From e1839f512319c9ad58b40c68c3cfed27fcbf55a9 Mon Sep 17 00:00:00 2001 From: Ivan Balashov Date: Tue, 19 Aug 2014 19:44:42 +0400 Subject: [PATCH 001/330] Performance optimizations TimestampedMessageParser made thread safe, removed locking --- .gitignore | 41 +++++++++++++++++++ .../pinterest/secor/common/SecorConfig.java | 33 ++++++++++----- .../parser/TimestampedMessageParser.java | 15 +++++-- .../pinterest/secor/writer/MessageWriter.java | 5 ++- 4 files changed, 79 insertions(+), 15 deletions(-) create mode 100644 .gitignore diff --git a/.gitignore b/.gitignore new file mode 100644 index 000000000..d46181d13 --- /dev/null +++ b/.gitignore @@ -0,0 +1,41 @@ +# 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 4362d9910..4a6db18e3 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -31,18 +31,31 @@ 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"); - - PropertiesConfiguration properties = new PropertiesConfiguration(configProperty); - - for (final Map.Entry entry : systemProperties.entrySet()) { - properties.setProperty(entry.getKey().toString(), entry.getValue()); + 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); + } + + 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) { diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 247a29943..299ae9f76 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -24,6 +24,17 @@ import java.util.TimeZone; public abstract class TimestampedMessageParser extends MessageParser { + + private static final ThreadLocal mFormatter = new ThreadLocal(){ + @Override + protected SimpleDateFormat initialValue() + { + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd"); + format.setTimeZone(TimeZone.getTimeZone("UTC")); + return format; + } + }; + public TimestampedMessageParser(SecorConfig config) { super(config); } @@ -49,9 +60,7 @@ 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)}; + String result[] = {"dt=" + mFormatter.get().format(date)}; return result; } } diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index f757365fb..c5fcf5b1e 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -44,6 +44,7 @@ public class MessageWriter { private FileRegistry mFileRegistry; private String mFileExtension; private CompressionCodec mCodec; + private String mLocalPrefix; public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry) throws Exception { @@ -57,6 +58,7 @@ public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, } else { mFileExtension = ""; } + mLocalPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); } private void adjustOffset(ParsedMessage message) throws IOException { @@ -79,8 +81,7 @@ public void write(ParsedMessage message) throws IOException { 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); + LogFilePath path = new LogFilePath(mLocalPrefix, mConfig.getGeneration(), offset, message, mFileExtension); LongWritable key = new LongWritable(message.getOffset()); BytesWritable value = new BytesWritable(message.getPayload()); SequenceFile.Writer writer; From 92b413f56b7ee1da9adfb9ed1251d6a4feb36f74 Mon Sep 17 00:00:00 2001 From: Andy Kramolisch Date: Tue, 12 Aug 2014 11:28:21 -0700 Subject: [PATCH 002/330] [secor][progress-monitor] Add option to print metrics as JSON to stdout --- README.md | 2 + src/main/config/secor.common.properties | 9 ++ .../pinterest/secor/common/KafkaClient.java | 23 ++-- .../pinterest/secor/common/SecorConfig.java | 12 ++ .../parser/TimestampedMessageParser.java | 2 +- .../secor/tools/ProgressMonitor.java | 130 +++++++++++++++--- 6 files changed, 152 insertions(+), 26 deletions(-) diff --git a/README.md b/README.md index 79c46d15b..9f2b5aa1f 100644 --- a/README.md +++ b/README.md @@ -86,6 +86,8 @@ Progress monitor exports offset consumption lags per topic partition to [OpenTSD java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup.properties -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.ProgressMonitorMain ``` +Alternatively, you can tell the progress monitor to simply dump metrics, as JSON, to stdout. To do this, set `secor.progress_monitor.json=true`. + ## Detailed design Design details are available in [DESIGN.md](DESIGN.md). diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 80601c105..0762d52ea 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -99,3 +99,12 @@ message.timestamp.input.pattern= # To enable compression, set this to a valid compression codec, such as 'org.apache.hadoop.io.compress.GzipCodec'. secor.compression.codec= + +# Used to determine whether to dump progress monitor output as JSON to stdout +# If left as false, will export to configured tsdb. +secor.progress_monitor.json=false + +# 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 diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 3fe96a268..aa1b6dfde 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -36,6 +36,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -58,7 +59,7 @@ public KafkaClient(SecorConfig config) { private HostAndPort findLeader(TopicPartition topicPartition) { SimpleConsumer consumer = null; try { - LOG.info("looking up lader for topic " + topicPartition.getTopic() + " partition " + + LOG.info("looking up leader for topic " + topicPartition.getTopic() + " partition " + topicPartition.getPartition()); consumer = new SimpleConsumer(mConfig.getKafkaSeedBrokerHost(), mConfig.getKafkaSeedBrokerPort(), @@ -114,7 +115,7 @@ private Message getMessage(TopicPartition topicPartition, long offset, SimpleConsumer consumer) { LOG.info("fetching message topic " + topicPartition.getTopic() + " partition " + topicPartition.getPartition() + " offset " + offset); - final int MAX_MESSAGE_SIZE_BYTES = 100000; + final 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, @@ -126,13 +127,17 @@ private Message getMessage(TopicPartition topicPartition, long offset, throw new RuntimeException("Error fetching offset data. Reason: " + response.errorCode(topicPartition.getTopic(), topicPartition.getPartition())); } - MessageAndOffset messageAndOffset = response.messageSet( - topicPartition.getTopic(), topicPartition.getPartition()).iterator().next(); - ByteBuffer payload = messageAndOffset.message().payload(); - byte[] payloadBytes = new byte[payload.limit()]; - payload.get(payloadBytes); - return new Message(topicPartition.getTopic(), topicPartition.getPartition(), - messageAndOffset.offset(), payloadBytes); + Iterator messageSetIterator = response.messageSet(topicPartition.getTopic(), topicPartition.getPartition()).iterator(); + if (messageSetIterator.hasNext()) { + MessageAndOffset messageAndOffset = messageSetIterator.next(); + ByteBuffer payload = messageAndOffset.message().payload(); + byte[] payloadBytes = new byte[payload.limit()]; + payload.get(payloadBytes); + return new Message(topicPartition.getTopic(), topicPartition.getPartition(), + messageAndOffset.offset(), payloadBytes); + } else { + return null; + } } private SimpleConsumer createConsumer(TopicPartition topicPartition) { diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 4a6db18e3..2c6090c78 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -194,6 +194,14 @@ public String getCompressionCodec() { return getString("secor.compression.codec"); } + public boolean getProgressMonitorAsJson() { + return getBoolean("secor.progress_monitor.json"); + } + + public int getMaxMessageSizeBytes() { + return getInt("secor.max_message_size.bytes"); + } + private void checkProperty(String name) { if (!mProperties.containsKey(name)) { throw new RuntimeException("Failed to find required configuration option '" + @@ -218,4 +226,8 @@ private long getLong(String name) { private String[] getStringArray(String name) { return mProperties.getStringArray(name); } + + private boolean getBoolean(String name) { + return mProperties.getBoolean(name, false); + } } diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 299ae9f76..e0653ea74 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -39,7 +39,7 @@ public TimestampedMessageParser(SecorConfig config) { super(config); } - abstract protected long extractTimestampMillis(final Message message) throws Exception; + public abstract long extractTimestampMillis(final Message message) throws Exception; protected static long toMillis(final long timestamp) { final long nanosecondDivider = (long) Math.pow(10, 9 + 9); diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 3491db350..9a6627c6b 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -16,21 +16,30 @@ */ package com.pinterest.secor.tools; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; 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 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; @@ -44,13 +53,16 @@ public class ProgressMonitor { private SecorConfig mConfig; private ZookeeperConnector mZookeeperConnector; private KafkaClient mKafkaClient; - private ThriftMessageParser mThriftMessageParser; + private MessageParser mMessageParser; - 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); } private void makeRequest(String body) throws IOException { @@ -111,7 +123,21 @@ private void exportToTsdb(String metric, Map tags, String value) } public void exportStats() throws Exception { + List stats = getStats(); + if (mConfig.getProgressMonitorAsJson()) { + System.out.println(JSONArray.toJSONString(stats)); + } else { + for (Stat stat : stats) { + exportToTsdb(stat.getMetric(), stat.getTags(), stat.getValue()); + } + } + + } + + private List getStats() throws Exception { List topics = mZookeeperConnector.getCommittedOffsetTopics(); + List stats = Lists.newArrayList(); + for (String topic : topics) { if (topic.matches(mConfig.getTsdbBlacklistTopics()) || !topic.matches(mConfig.getKafkaTopicFilter())) { @@ -129,8 +155,7 @@ public void exportStats() throws Exception { partition); } else { committedOffset = committedMessage.getOffset(); - committedTimestampMillis = mThriftMessageParser.extractTimestampMillis( - committedMessage); + committedTimestampMillis = getTimestamp(committedMessage); } Message lastMessage = mKafkaClient.getLastMessage(topicPartition); @@ -138,18 +163,20 @@ public void exportStats() throws Exception { LOG.warn("no message found in topic " + topic + " partition " + 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)); + Map tags = ImmutableMap.of( + "topic", topic, + "partition", Integer.toString(partition) + ); + + stats.add(new Stat("secor.lag.offsets", tags, Long.toString(offsetLag))); + stats.add(new Stat("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 " + @@ -157,5 +184,76 @@ public void exportStats() throws Exception { } } } + + return stats; + } + + private long getTimestamp(Message message) throws Exception { + if (mMessageParser instanceof TimestampedMessageParser) { + return ((TimestampedMessageParser)mMessageParser).extractTimestampMillis(message); + } else { + return -1; + } + } + + private static class Stat { + final String metric; + final Map tags; + final String value; + + public Stat(String metric, Map tags, String value) + { + this.metric = metric; + this.tags = tags; + this.value = value; + } + + public String getMetric() { + return this.metric; + } + + public Map getTags() { + return this.tags; + } + + public String getValue() { + return this.value; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof Stat)) { + return false; + } + + Stat stat = (Stat) o; + + if (metric != null ? !metric.equals(stat.metric) : stat.metric != null) { + return false; + } + if (tags != null ? !tags.equals(stat.tags) : stat.tags != null) { + return false; + } + if (value != null ? !value.equals(stat.value) : stat.value != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = metric != null ? metric.hashCode() : 0; + result = 31 * result + (tags != null ? tags.hashCode() : 0); + result = 31 * result + (value != null ? value.hashCode() : 0); + return result; + } + + public String toString() {return "com.pinterest.secor.tools.ProgressMonitor.Stat(metric=" + this.metric + ", tags=" + this.tags + ", value=" + this.value + ")";} } } From efb6f78d4d918425a109927d04b936a82219981d Mon Sep 17 00:00:00 2001 From: Andy Kramolisch Date: Thu, 21 Aug 2014 14:30:58 -0700 Subject: [PATCH 003/330] [progress-monitor] Address feedback --- src/main/config/secor.common.properties | 2 +- .../pinterest/secor/common/KafkaClient.java | 22 ++++++++----------- .../pinterest/secor/common/SecorConfig.java | 2 +- .../secor/tools/ProgressMonitor.java | 19 +++++++++++++--- 4 files changed, 27 insertions(+), 18 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 0762d52ea..56a7fc12c 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -107,4 +107,4 @@ secor.progress_monitor.json=false # 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 +secor.max.message.size.bytes=100000 diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index aa1b6dfde..2f48e1123 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -18,7 +18,8 @@ import com.google.common.net.HostAndPort; import com.pinterest.secor.message.Message; -import kafka.api.*; +import kafka.api.FetchRequestBuilder; +import kafka.api.PartitionOffsetRequestInfo; import kafka.common.TopicAndPartition; import kafka.javaapi.FetchResponse; import kafka.javaapi.OffsetRequest; @@ -36,7 +37,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; @@ -127,17 +127,13 @@ private Message getMessage(TopicPartition topicPartition, long offset, throw new RuntimeException("Error fetching offset data. Reason: " + response.errorCode(topicPartition.getTopic(), topicPartition.getPartition())); } - Iterator messageSetIterator = response.messageSet(topicPartition.getTopic(), topicPartition.getPartition()).iterator(); - if (messageSetIterator.hasNext()) { - MessageAndOffset messageAndOffset = messageSetIterator.next(); - ByteBuffer payload = messageAndOffset.message().payload(); - byte[] payloadBytes = new byte[payload.limit()]; - payload.get(payloadBytes); - return new Message(topicPartition.getTopic(), topicPartition.getPartition(), - messageAndOffset.offset(), payloadBytes); - } else { - return null; - } + MessageAndOffset messageAndOffset = response.messageSet( + topicPartition.getTopic(), topicPartition.getPartition()).iterator().next(); + ByteBuffer payload = messageAndOffset.message().payload(); + byte[] payloadBytes = new byte[payload.limit()]; + payload.get(payloadBytes); + return new Message(topicPartition.getTopic(), topicPartition.getPartition(), + messageAndOffset.offset(), payloadBytes); } private SimpleConsumer createConsumer(TopicPartition topicPartition) { diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 2c6090c78..243b80491 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -199,7 +199,7 @@ public boolean getProgressMonitorAsJson() { } public int getMaxMessageSizeBytes() { - return getInt("secor.max_message_size.bytes"); + return getInt("secor.max.message.size.bytes"); } private void checkProperty(String name) { diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 9a6627c6b..07464098e 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -16,7 +16,6 @@ */ package com.pinterest.secor.tools; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.pinterest.secor.common.KafkaClient; @@ -131,7 +130,6 @@ public void exportStats() throws Exception { exportToTsdb(stat.getMetric(), stat.getTags(), stat.getValue()); } } - } private List getStats() throws Exception { @@ -200,12 +198,14 @@ private static class Stat { final String metric; final Map tags; final String value; + final long timeStamp; public Stat(String metric, Map tags, String value) { this.metric = metric; this.tags = tags; this.value = value; + this.timeStamp = System.currentTimeMillis() / 1000; } public String getMetric() { @@ -254,6 +254,19 @@ public int hashCode() return result; } - public String toString() {return "com.pinterest.secor.tools.ProgressMonitor.Stat(metric=" + this.metric + ", tags=" + this.tags + ", value=" + this.value + ")";} + public String toString() { + JSONObject bodyJson = new JSONObject(); + bodyJson.put("metric", metric); + bodyJson.put("timestamp", timeStamp); + 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); + return bodyJson.toString(); + } } } From 3be8f69bc75f3774f34958d49fd3d93d25564f1c Mon Sep 17 00:00:00 2001 From: Ivan Balashov Date: Thu, 21 Aug 2014 19:56:49 +0400 Subject: [PATCH 004/330] fixed offsets storage path in zk --- .../pinterest/secor/common/SecorConfig.java | 2 +- .../secor/common/ZookeeperConnector.java | 25 ++++++++++++++-- .../secor/common/ZookeeperConnectorTest.java | 30 +++++++++++++++++++ 3 files changed, 54 insertions(+), 3 deletions(-) create mode 100644 src/test/java/com/pinterest/secor/common/ZookeeperConnectorTest.java diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 4a6db18e3..56d6d9a84 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -58,7 +58,7 @@ public static SecorConfig load() throws ConfigurationException { return mSecorConfig.get(); } - private SecorConfig(PropertiesConfiguration properties) { + protected SecorConfig(PropertiesConfiguration properties) { mProperties = properties; } diff --git a/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java b/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java index 20c3e56d0..62287c94c 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) { @@ -179,4 +196,8 @@ public void deleteCommittedOffsetPartitionCount(TopicPartition topicPartition) LOG.info("deleting path " + offsetPath); zookeeper.delete(offsetPath, -1); } + + protected void setConfig(SecorConfig config) { + this.mConfig = config; + } } 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 From e109dfc6bbb3283d16bdcf81ca33f38a84e3968c Mon Sep 17 00:00:00 2001 From: Andy Kramolisch Date: Tue, 26 Aug 2014 11:21:57 -0700 Subject: [PATCH 005/330] [progress-monitor] Address feedback --- src/main/config/secor.common.properties | 4 - .../pinterest/secor/common/SecorConfig.java | 10 +- .../secor/tools/ProgressMonitor.java | 106 ++++-------------- 3 files changed, 20 insertions(+), 100 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 56a7fc12c..ba1b07afe 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -100,10 +100,6 @@ message.timestamp.input.pattern= # To enable compression, set this to a valid compression codec, such as 'org.apache.hadoop.io.compress.GzipCodec'. secor.compression.codec= -# Used to determine whether to dump progress monitor output as JSON to stdout -# If left as false, will export to configured tsdb. -secor.progress_monitor.json=false - # 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 243b80491..0224a34d7 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -185,7 +185,7 @@ public String getTsdbBlacklistTopics() { public String getMessageTimestampName() { return getString("message.timestamp.name"); } - + public String getMessageTimestampInputPattern() { return getString("message.timestamp.input.pattern"); } @@ -194,10 +194,6 @@ public String getCompressionCodec() { return getString("secor.compression.codec"); } - public boolean getProgressMonitorAsJson() { - return getBoolean("secor.progress_monitor.json"); - } - public int getMaxMessageSizeBytes() { return getInt("secor.max.message.size.bytes"); } @@ -226,8 +222,4 @@ private long getLong(String name) { private String[] getStringArray(String name) { return mProperties.getStringArray(name); } - - private boolean getBoolean(String name) { - return mProperties.getBoolean(name, false); - } } diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 07464098e..438b409c4 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -106,28 +106,18 @@ 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(); - if (mConfig.getProgressMonitorAsJson()) { - System.out.println(JSONArray.toJSONString(stats)); - } else { + System.out.println(JSONArray.toJSONString(stats)); + if (mConfig.getTsdbHostport() != null && !mConfig.getTsdbHostport().isEmpty()) { for (Stat stat : stats) { - exportToTsdb(stat.getMetric(), stat.getTags(), stat.getValue()); + exportToTsdb(stat); } } } @@ -172,8 +162,9 @@ private List getStats() throws Exception { "partition", Integer.toString(partition) ); - stats.add(new Stat("secor.lag.offsets", tags, Long.toString(offsetLag))); - stats.add(new Stat("secor.lag.seconds", tags, Long.toString(timestampMillisLag / 1000))); + long timestamp = System.currentTimeMillis() / 1000; + stats.add(Stat.createInstance("secor.lag.offsets", tags, Long.toString(offsetLag), timestamp)); + stats.add(Stat.createInstance("secor.lag.seconds", tags, Long.toString(timestampMillisLag / 1000), timestamp)); LOG.debug("topic " + topic + " partition " + partition + " committed offset " + committedOffset + " last offset " + lastOffset + " committed timestamp " + @@ -194,79 +185,20 @@ private long getTimestamp(Message message) throws Exception { } } - private static class Stat { - final String metric; - final Map tags; - final String value; - final long timeStamp; - - public Stat(String metric, Map tags, String value) - { - this.metric = metric; - this.tags = tags; - this.value = value; - this.timeStamp = System.currentTimeMillis() / 1000; - } - - public String getMetric() { - return this.metric; - } - - public Map getTags() { - return this.tags; - } - - public String getValue() { - return this.value; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (!(o instanceof Stat)) { - return false; - } - - Stat stat = (Stat) o; - - if (metric != null ? !metric.equals(stat.metric) : stat.metric != null) { - return false; - } - if (tags != null ? !tags.equals(stat.tags) : stat.tags != null) { - return false; - } - if (value != null ? !value.equals(stat.value) : stat.value != null) { - return false; - } - - return true; - } + private static class Stat extends JSONObject { - @Override - public int hashCode() + public static Stat createInstance(String metric, Map tags, String value, long timestamp) { - int result = metric != null ? metric.hashCode() : 0; - result = 31 * result + (tags != null ? tags.hashCode() : 0); - result = 31 * result + (value != null ? value.hashCode() : 0); - return result; + return new Stat(ImmutableMap.of( + "metric", metric, + "tags", tags, + "value", value, + "timestamp", timestamp + )); } - public String toString() { - JSONObject bodyJson = new JSONObject(); - bodyJson.put("metric", metric); - bodyJson.put("timestamp", timeStamp); - 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); - return bodyJson.toString(); + public Stat(Map map) { + super(map); } } } From 076560542e5c29f0cb01e06b775f00343282068f Mon Sep 17 00:00:00 2001 From: Andy Kramolisch Date: Thu, 28 Aug 2014 11:25:32 -0700 Subject: [PATCH 006/330] [progress-monitor] Address feedback --- README.md | 2 -- src/main/java/com/pinterest/secor/tools/ProgressMonitor.java | 3 +-- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/README.md b/README.md index 9f2b5aa1f..79c46d15b 100644 --- a/README.md +++ b/README.md @@ -86,8 +86,6 @@ Progress monitor exports offset consumption lags per topic partition to [OpenTSD java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup.properties -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.ProgressMonitorMain ``` -Alternatively, you can tell the progress monitor to simply dump metrics, as JSON, to stdout. To do this, set `secor.progress_monitor.json=true`. - ## Detailed design Design details are available in [DESIGN.md](DESIGN.md). diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 438b409c4..f7ea27d0b 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -187,8 +187,7 @@ private long getTimestamp(Message message) throws Exception { private static class Stat extends JSONObject { - public static Stat createInstance(String metric, Map tags, String value, long timestamp) - { + public static Stat createInstance(String metric, Map tags, String value, long timestamp) { return new Stat(ImmutableMap.of( "metric", metric, "tags", tags, From f8721bf468f1c60e07dfa56db560c186f6e98121 Mon Sep 17 00:00:00 2001 From: James Green Date: Tue, 16 Sep 2014 14:56:39 +0100 Subject: [PATCH 007/330] Modified consumer to adjust seen offset for all messages, regardless of whether a message is parseable. --- src/main/java/com/pinterest/secor/consumer/Consumer.java | 6 ++++++ src/main/java/com/pinterest/secor/writer/MessageWriter.java | 4 ++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 73c3056f9..1cd80f0c1 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -93,6 +93,12 @@ public void run() { LOG.trace("Consumer timed out", e); } if (rawMessage != null) { + // Before parsing, update the offset and remove any redundant data + try { + mMessageWriter.adjustOffset(rawMessage); + } catch (IOException e) { + throw new RuntimeException("Failed to adjust offset.", e); + } ParsedMessage parsedMessage = null; try { parsedMessage = mMessageParser.parse(rawMessage); diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index c5fcf5b1e..1f5ef2944 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -18,6 +18,7 @@ import com.pinterest.secor.common.*; import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; import java.io.IOException; @@ -61,7 +62,7 @@ public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, mLocalPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); } - 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); @@ -77,7 +78,6 @@ private void adjustOffset(ParsedMessage message) throws IOException { } public void write(ParsedMessage message) throws IOException { - adjustOffset(message); TopicPartition topicPartition = new TopicPartition(message.getTopic(), message.getKafkaPartition()); long offset = mOffsetTracker.getAdjustedCommittedOffsetCount(topicPartition); From 2c60623054a32fc809bb2749770da17e28c33c8b Mon Sep 17 00:00:00 2001 From: jfgreen Date: Wed, 17 Sep 2014 10:46:37 +0100 Subject: [PATCH 008/330] Update contributors. --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 79c46d15b..4e2dbb676 100644 --- a/README.md +++ b/README.md @@ -101,6 +101,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Andy Kramolisch](https://github.com/andykram) * [Brenden Matthews](https://github.com/brndnmtthws) * [Lucas Zago](https://github.com/zago) + * [James Green] (https://github.com/jfgreen) ## Help From 756babbba3e689aaba0becade8a60d52297256c9 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Fri, 19 Sep 2014 10:29:03 -0700 Subject: [PATCH 009/330] gzip reader and writer --- .gitignore | 4 + src/main/config/secor.common.properties | 6 + .../pinterest/secor/common/FileRegistry.java | 40 ++--- .../pinterest/secor/common/SecorConfig.java | 15 +- .../pinterest/secor/consumer/Consumer.java | 6 +- .../com/pinterest/secor/io/FileReader.java | 42 +++++ .../pinterest/secor/io/FileReaderFactory.java | 45 +++++ .../com/pinterest/secor/io/FileWriter.java | 51 ++++++ .../pinterest/secor/io/FileWriterFactory.java | 41 +++++ .../java/com/pinterest/secor/io/KeyValue.java | 45 +++++ .../secor/io/impl/GzipFileReader.java | 70 ++++++++ .../secor/io/impl/GzipFileWriter.java | 66 +++++++ .../secor/io/impl/SequenceFileReader.java | 70 ++++++++ .../secor/io/impl/SequenceFileWriter.java | 76 ++++++++ .../pinterest/secor/uploader/Uploader.java | 40 ++--- .../pinterest/secor/util/ReflectionUtil.java | 87 +++++++--- .../pinterest/secor/writer/MessageWriter.java | 31 ++-- .../secor/common/FileRegistryTest.java | 83 +++------ .../pinterest/secor/io/FileReaderTest.java | 134 +++++++++++++++ .../pinterest/secor/io/FileWriterTest.java | 162 ++++++++++++++++++ .../secor/uploader/UploaderTest.java | 42 ++--- 21 files changed, 980 insertions(+), 176 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/io/FileReader.java create mode 100644 src/main/java/com/pinterest/secor/io/FileReaderFactory.java create mode 100644 src/main/java/com/pinterest/secor/io/FileWriter.java create mode 100644 src/main/java/com/pinterest/secor/io/FileWriterFactory.java create mode 100644 src/main/java/com/pinterest/secor/io/KeyValue.java create mode 100644 src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java create mode 100644 src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java create mode 100644 src/main/java/com/pinterest/secor/io/impl/SequenceFileReader.java create mode 100644 src/main/java/com/pinterest/secor/io/impl/SequenceFileWriter.java create mode 100644 src/test/java/com/pinterest/secor/io/FileReaderTest.java create mode 100644 src/test/java/com/pinterest/secor/io/FileWriterTest.java diff --git a/.gitignore b/.gitignore index d46181d13..f1e123720 100644 --- a/.gitignore +++ b/.gitignore @@ -39,3 +39,7 @@ Thumbs.db *.iml .idea target +bin/ +.settings/ +.project +.classpath diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index ba1b07afe..36e061fde 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -100,6 +100,12 @@ message.timestamp.input.pattern= # To enable compression, set this to a valid compression codec, such as 'org.apache.hadoop.io.compress.GzipCodec'. secor.compression.codec= +# The secor file writer used to write the data, by default we write sequence files +secor.file.writer=com.pinterest.secor.io.impl.SequenceFileWriter + +#The secor file reader used to read the data, by default we write/read sequence files +secor.file.reader=com.pinterest.secor.io.impl.SequenceFileReader + # 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 diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index f405cc28d..f776e9ec2 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.io.FileWriterFactory; import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.StatsUtil; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.SequenceFile; + import org.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 final SecorConfig mConfig; private HashMap> mFiles; - private HashMap mWriters; + private HashMap mWriters; private HashMap mCreationTimes; - public FileRegistry() { + public FileRegistry(SecorConfig mConfig) { + this.mConfig = mConfig; mFiles = new HashMap>(); - mWriters = new HashMap(); + mWriters = new HashMap(); mCreationTimes = new HashMap(); } @@ -81,10 +80,10 @@ public Collection getPaths(TopicPartition topicPartition) { * @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()); @@ -99,18 +98,7 @@ 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 = FileWriterFactory.create(path, codec, this.mConfig); mWriters.put(path, writer); mCreationTimes.put(path, System.currentTimeMillis() / 1000L); LOG.debug("created writer for path " + path.getLogFilePath()); @@ -161,7 +149,7 @@ 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()); } else { @@ -199,7 +187,7 @@ public long getSize(TopicPartition topicPartition) throws IOException { Collection paths = getPaths(topicPartition); long result = 0; for (LogFilePath path : paths) { - SequenceFile.Writer writer = mWriters.get(path); + FileWriter writer = mWriters.get(path); if (writer != null) { result += writer.getLength(); } diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index ec406c6b7..b11e5d16f 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -58,7 +58,12 @@ public static SecorConfig load() throws ConfigurationException { return mSecorConfig.get(); } - protected SecorConfig(PropertiesConfiguration properties) { + /** + * Exposed for testability + * + * @param properties + */ + public SecorConfig(PropertiesConfiguration properties) { mProperties = properties; } @@ -197,6 +202,14 @@ public String getCompressionCodec() { public int getMaxMessageSizeBytes() { return getInt("secor.max.message.size.bytes"); } + + public String getFileWriter() { + return getString("secor.file.writer"); + } + + public String getFileReader() { + return getString("secor.file.reader"); + } private void checkProperty(String name) { if (!mProperties.containsKey(name)) { diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 1cd80f0c1..68857d837 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -26,7 +26,9 @@ import com.pinterest.secor.reader.MessageReader; import com.pinterest.secor.util.ReflectionUtil; import com.pinterest.secor.writer.MessageWriter; + import kafka.consumer.ConsumerTimeoutException; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,7 +64,7 @@ public Consumer(SecorConfig config) { private void init() throws Exception { OffsetTracker offsetTracker = new OffsetTracker(); mMessageReader = new MessageReader(mConfig, offsetTracker); - FileRegistry fileRegistry = new FileRegistry(); + FileRegistry fileRegistry = new FileRegistry(mConfig); mMessageWriter = new MessageWriter(mConfig, offsetTracker, fileRegistry); mMessageParser = (MessageParser) ReflectionUtil.createMessageParser( mConfig.getMessageParserClass(), mConfig); @@ -116,7 +118,7 @@ public void run() { if (parsedMessage != null) { try { mMessageWriter.write(parsedMessage); - } catch (IOException e) { + } catch (Exception e) { throw new RuntimeException("Failed to write message " + parsedMessage, e); } } 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..f96b8e213 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/FileReader.java @@ -0,0 +1,42 @@ +/** + * 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 all secor files + * + * @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; + +} diff --git a/src/main/java/com/pinterest/secor/io/FileReaderFactory.java b/src/main/java/com/pinterest/secor/io/FileReaderFactory.java new file mode 100644 index 000000000..0e71cf8ab --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/FileReaderFactory.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.io; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.util.ReflectionUtil; + +/** + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class FileReaderFactory { + + /** + * Given a log file path create a file reader using the file reader configured + * + * @param path + * @param mConfig + * @return + * @throws Exception + */ + public static FileReader create(LogFilePath path, SecorConfig mConfig) throws Exception { + if (mConfig.getFileReader() != null && !mConfig.getFileReader().isEmpty()) { + return ((FileReader) ReflectionUtil.createFileReader(mConfig.getFileReader(), path)); + } + throw new IllegalArgumentException("File Reader not defined or empty"); + } + +} 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..0df0ea7eb --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/FileWriter.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.io; + +import java.io.IOException; + +/** + * Generic file writer interface for all secor files + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public interface FileWriter { + + /** + * Close the file + * @throws IOException + */ + public void close() throws IOException; + + /** + * Get length of data written up to now to the underlying file + * @return + * @throws IOException + */ + public long getLength() throws IOException; + + /** + * Write the given key and value to the file + * + * @param key + * @param value + * @throws IOException + */ + public void write(long key, byte[] value) throws IOException; + +} diff --git a/src/main/java/com/pinterest/secor/io/FileWriterFactory.java b/src/main/java/com/pinterest/secor/io/FileWriterFactory.java new file mode 100644 index 000000000..cee5817d2 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/FileWriterFactory.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.io; + + +import org.apache.hadoop.io.compress.CompressionCodec; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.util.ReflectionUtil; + +/** + * File Writer Factory which returns a writer based upon secor configuration + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class FileWriterFactory { + + public static FileWriter create(LogFilePath path, CompressionCodec codec, SecorConfig mConfig) throws Exception { + if (mConfig.getFileWriter() != null && !mConfig.getFileWriter().isEmpty()) { + return ((FileWriter) ReflectionUtil.createFileWriter(mConfig.getFileWriter(), path, codec)); + } + throw new IllegalArgumentException("File Writer not defined or empty"); + } + +} 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..b4ba8fa63 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/KeyValue.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.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 key; + private final byte[] value; + + //constructor + public KeyValue(long key, byte[] value) { + this.key = key; + this.value = value; + } + + public long getKey() { + return this.key; + } + + public byte[] getValue() { + return this.value; + } + +} diff --git a/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java b/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java new file mode 100644 index 000000000..7ff508dce --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.io.impl; + +import java.io.ByteArrayOutputStream; +import java.io.EOFException; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.zip.GZIPInputStream; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.KeyValue; + +/** + * Gzip File reader implementation + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class GzipFileReader implements FileReader { + + private static final byte DELIMITER = '\n'; + + private final GZIPInputStream reader; + private long offset; + + // constructor + public GzipFileReader(LogFilePath path) throws Exception { + reader = new GZIPInputStream(new FileInputStream(path.getLogFilePath())); + this.offset = path.getOffset(); + } + + @Override + public KeyValue next() throws IOException { + ByteArrayOutputStream messageBuffer = new ByteArrayOutputStream(); + int nextByte; + while ((nextByte = reader.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.offset++, messageBuffer.toByteArray()); + } + + @Override + public void close() throws IOException { + this.reader.close(); + } + +} diff --git a/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java b/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java new file mode 100644 index 000000000..18e9c37b9 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java @@ -0,0 +1,66 @@ +/** + * 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.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.zip.GZIPOutputStream; + +import com.google.common.io.CountingOutputStream; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.FileWriter; + +/** + * + * Gzip File writer implementation + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class GzipFileWriter implements FileWriter { + + //delimiter used between messages + private static final byte DELIMITER = '\n'; + + private final CountingOutputStream countingStream; + private final GZIPOutputStream writer; + + // constructor + public GzipFileWriter(LogFilePath path) throws FileNotFoundException, + IOException { + this.countingStream = new CountingOutputStream(new FileOutputStream(path.getLogFilePath())); + this.writer = new GZIPOutputStream(this.countingStream); + } + + @Override + public void close() throws IOException { + this.writer.close(); + } + + @Override + public long getLength() throws IOException { + return this.countingStream.getCount(); + } + + @Override + public void write(long key, byte[] value) throws IOException { + this.writer.write(value); + this.writer.write(DELIMITER); + } + +} diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReader.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReader.java new file mode 100644 index 000000000..afe40a9cc --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReader.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.io.impl; + +import java.io.IOException; + +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.common.LogFilePath; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.KeyValue; + +/** + * + * Sequence file reader implementation + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class SequenceFileReader implements FileReader { + + private final SequenceFile.Reader reader; + private final LongWritable key; + private final BytesWritable value; + + // constructor + public SequenceFileReader(LogFilePath path) throws Exception { + Configuration config = new Configuration(); + FileSystem fs = FileSystem.get(config); + String srcFilename = path.getLogFilePath(); + Path srcFsPath = new Path(srcFilename); + this.reader = new SequenceFile.Reader(fs, srcFsPath, config); + this.key = (LongWritable) reader.getKeyClass().newInstance(); + this.value = (BytesWritable) reader.getValueClass().newInstance(); + } + + @Override + public KeyValue next() throws IOException { + if(reader.next(key, value)) { + return new KeyValue(key.get(), value.getBytes()); + } else { + return null; + } + } + + @Override + public void close() throws IOException { + this.reader.close(); + } + +} diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileWriter.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileWriter.java new file mode 100644 index 000000000..cb7c5cf96 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileWriter.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.io.impl; + +import java.io.IOException; + +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 com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.FileWriter; + +/** + * + * Sequence file writer implementation + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class SequenceFileWriter implements FileWriter { + + private final SequenceFile.Writer writer; + + // constructor + public SequenceFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { + Configuration config = new Configuration(); + FileSystem fs = FileSystem.get(config); + if (codec != null) { + Path fsPath = new Path(path.getLogFilePath()); + this.writer = SequenceFile.createWriter(fs, config, fsPath, LongWritable.class, + BytesWritable.class, + SequenceFile.CompressionType.BLOCK, codec); + } else { + Path fsPath = new Path(path.getLogFilePath()); + this.writer = SequenceFile.createWriter(fs, config, fsPath, LongWritable.class, + BytesWritable.class); + } + } + + @Override + public void close() throws IOException { + this.writer.close(); + } + + @Override + public long getLength() throws IOException { + return this.writer.getLength(); + } + + @Override + public void write(long key, byte[] value) throws IOException { + LongWritable writeableKey = new LongWritable(key); + BytesWritable writeableValue = new BytesWritable(value); + this.writer.append(writeableKey, writeableValue); + } + +} diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index 0c12e4153..36c94818e 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -17,19 +17,18 @@ package com.pinterest.secor.uploader; import com.pinterest.secor.common.*; -import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileReaderFactory; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.util.FileUtil; 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.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.*; /** @@ -102,38 +101,33 @@ 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) throws Exception { + return FileReaderFactory.create(srcPath, this.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(); + reader = createReader(srcPath); CompressionCodec codec = null; String extension = ""; if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { codec = (CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec()); extension = codec.getDefaultExtension(); } - while (reader.next(key, value)) { - if (key.get() >= startOffset) { + KeyValue keyVal; + while ((keyVal = reader.next()) != null) { + if (keyVal.getKey() >= startOffset) { if (writer == null) { String localPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); @@ -142,7 +136,7 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { srcPath.getKafkaPartition(), startOffset, extension); writer = mFileRegistry.getOrCreateWriter(dstPath, codec); } - writer.append(key, value); + writer.write(keyVal.getKey(), keyVal.getValue()); copiedMessages++; } } @@ -155,7 +149,7 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { if (dstPath == null) { LOG.info("removed file " + srcPath.getLogFilePath()); } else { - LOG.info("trimmed " + copiedMessages + " messages from " + srcFilename + " to " + + LOG.info("trimmed " + copiedMessages + " messages from " + srcPath.getLogFilePath() + " to " + dstPath.getLogFilePath() + " with start offset " + startOffset); } } diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 8cad96d4a..bde40df0b 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -16,35 +16,78 @@ */ package com.pinterest.secor.util; +import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.common.SecorConfig; 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) */ public class ReflectionUtil { - public static Object createCompressionCodec(String className) throws Exception { - Class clazz = Class.forName(className); - return clazz.getConstructor().newInstance(); - } - - public static Object createMessageParser(String className, - SecorConfig config) throws Exception { - Class clazz = Class.forName(className); - - // Search for an "appropriate" constructor. - for (Constructor ctor : clazz.getConstructors()) { - Class[] paramTypes = ctor.getParameterTypes(); - - // If the arity matches, let's use it. - if (paramTypes.length == 1) { - Object[] args = {config}; - return ctor.newInstance(args); - } - } - throw new IllegalArgumentException("Class not found " + className); - } + public static Object createCompressionCodec(String className) + throws Exception { + Class clazz = Class.forName(className); + return clazz.getConstructor().newInstance(); + } + + public static Object createMessageParser(String className, + SecorConfig config) throws Exception { + Class clazz = Class.forName(className); + + // Search for an "appropriate" constructor. + for (Constructor ctor : clazz.getConstructors()) { + Class[] paramTypes = ctor.getParameterTypes(); + + // If the arity matches, let's use it. + if (paramTypes.length == 1) { + Object[] args = { config }; + return ctor.newInstance(args); + } + } + throw new IllegalArgumentException("Class not found " + className); + } + + public static Object createFileWriter(String className, + LogFilePath logFilePath, CompressionCodec compressionCodec) + throws Exception { + Class clazz = Class.forName(className); + // Search for an "appropriate" constructor. + for (Constructor ctor : clazz.getConstructors()) { + Class[] paramTypes = ctor.getParameterTypes(); + + // If the arity matches, let's use it. + if (paramTypes.length == 1) { + Object[] args = { logFilePath }; + return ctor.newInstance(args); + } else if (paramTypes.length == 2) { + Object[] args = { logFilePath, compressionCodec }; + return ctor.newInstance(args); + } + } + throw new IllegalArgumentException("Class not found " + className); + + } + + public static Object createFileReader(String className, LogFilePath path) + throws Exception { + Class clazz = Class.forName(className); + + // Search for an "appropriate" constructor. + for (Constructor ctor : clazz.getConstructors()) { + Class[] paramTypes = ctor.getParameterTypes(); + + // If the arity matches, let's use it. + if (paramTypes.length == 1) { + Object[] args = { path }; + return ctor.newInstance(args); + } + } + throw new IllegalArgumentException("Class not found " + className); + } } diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index 1f5ef2944..7c582d57f 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -17,6 +17,7 @@ package com.pinterest.secor.writer; import com.pinterest.secor.common.*; +import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; @@ -24,11 +25,6 @@ import java.io.IOException; 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 org.apache.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,8 +39,7 @@ public class MessageWriter { private SecorConfig mConfig; private OffsetTracker mOffsetTracker; private FileRegistry mFileRegistry; - private String mFileExtension; - private CompressionCodec mCodec; + private LogFilePathAttributes mFilePathAttributes; private String mLocalPrefix; public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, @@ -52,13 +47,7 @@ public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, mConfig = config; mOffsetTracker = offsetTracker; mFileRegistry = fileRegistry; - if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { - mCodec = - ((CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec())); - mFileExtension = mCodec.getDefaultExtension(); - } else { - mFileExtension = ""; - } + mFilePathAttributes = new LogFilePathAttributes(config); mLocalPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); } @@ -77,16 +66,16 @@ public void adjustOffset(Message message) throws IOException { mOffsetTracker.setLastSeenOffset(topicPartition, message.getOffset()); } - public void write(ParsedMessage message) throws IOException { + public void write(ParsedMessage message) throws Exception { + adjustOffset(message); TopicPartition topicPartition = new TopicPartition(message.getTopic(), message.getKafkaPartition()); long offset = mOffsetTracker.getAdjustedCommittedOffsetCount(topicPartition); - LogFilePath path = new LogFilePath(mLocalPrefix, 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); + LogFilePath path = new LogFilePath(mLocalPrefix, mConfig.getGeneration(), offset, message, + mFilePathAttributes.getLogFileExtension()); + FileWriter writer; + writer = mFileRegistry.getOrCreateWriter(path, mFilePathAttributes.getCompressionCodec()); + writer.write(message.getOffset(), message.getPayload()); LOG.debug("appended message " + message + " to file " + path.getLogFilePath() + ". File length " + writer.getLength()); } diff --git a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java index 897f0605a..1f893ed92 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.io.FileWriterFactory; import com.pinterest.secor.util.FileUtil; + import junit.framework.TestCase; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.SequenceFile; + +import org.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,7 +39,7 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ @RunWith(PowerMockRunner.class) -@PrepareForTest({FileRegistry.class, FileSystem.class, FileUtil.class, SequenceFile.class}) +@PrepareForTest({FileRegistry.class, FileUtil.class, FileWriterFactory.class}) public class FileRegistryTest extends TestCase { private static final String PATH = "/some_parent_dir/some_topic/some_partition/some_other_partition/" + @@ -58,32 +57,27 @@ public class FileRegistryTest extends TestCase { public void setUp() throws Exception { super.setUp(); + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.addProperty("secor.file.writer", "com.pinterest.secor.io.impl.SequenceFileWriter"); + 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 void 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( + PowerMockito.mockStatic(FileWriterFactory.class); + FileWriter writer = Mockito.mock(FileWriter.class); + Mockito.when(FileWriterFactory.create(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); } @@ -95,19 +89,14 @@ public void testGetOrCreateWriter() throws Exception { // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); - FileSystem.get(Mockito.any(Configuration.class)); + FileWriterFactory.create(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(); assertEquals(1, topicPartitions.size()); @@ -118,28 +107,18 @@ public void testGetOrCreateWriter() throws Exception { assertTrue(logFilePaths.contains(mLogFilePath)); } - private void createCompressedWriter() throws IOException { + 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( + PowerMockito.mockStatic(FileWriterFactory.class); + FileWriter writer = Mockito.mock(FileWriter.class); + Mockito.when(FileWriterFactory.create(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,22 +128,14 @@ 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) - ); + FileWriterFactory.create(Mockito.any(LogFilePath.class), + Mockito.any(CompressionCodec.class), Mockito.any(SecorConfig.class)); TopicPartition topicPartition = new TopicPartition("some_topic", 0); Collection topicPartitions = mRegistry.getTopicPartitions(); diff --git a/src/test/java/com/pinterest/secor/io/FileReaderTest.java b/src/test/java/com/pinterest/secor/io/FileReaderTest.java new file mode 100644 index 000000000..36c5e08dd --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/FileReaderTest.java @@ -0,0 +1,134 @@ +/** + * 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.FileInputStream; +import java.io.InputStream; +import java.util.zip.GZIPInputStream; + +import org.apache.commons.configuration.PropertiesConfiguration; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.impl.GzipFileReader; +import com.pinterest.secor.io.impl.SequenceFileReader; + +import junit.framework.TestCase; + +/** + * Test the file readers + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ FileSystem.class, GzipFileReader.class, SequenceFileReader.class }) +public class FileReaderTest 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 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", + "com.pinterest.secor.io.impl.SequenceFileReader"); + mConfig = new SecorConfig(properties); + } + + private void setupGzipFileReaderConfig() { + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.addProperty("secor.file.reader", + "com.pinterest.secor.io.impl.GzipFileReader"); + mConfig = new SecorConfig(properties); + } + + public void testSequenceFileReader() throws Exception { + setupSequenceFileReaderConfig(); + mockSequenceFileReader(); + + FileReaderFactory.create(mLogFilePath, mConfig); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem.get(Mockito.any(Configuration.class)); + } + + public void testGzipFileReader() throws Exception { + setupGzipFileReaderConfig(); + mockGzipFileReader(); + FileReaderFactory.create(mLogFilePathGz, mConfig); + } + + private void mockGzipFileReader() throws Exception { + FileInputStream fileInputStreamMock = PowerMockito + .mock(FileInputStream.class); + + GZIPInputStream gzipInputStreamMock = PowerMockito + .mock(GZIPInputStream.class); + + PowerMockito.whenNew(FileInputStream.class) + .withParameterTypes(String.class) + .withArguments(Mockito.any(String.class)) + .thenReturn(fileInputStreamMock); + + PowerMockito.whenNew(GZIPInputStream.class) + .withParameterTypes(InputStream.class) + .withArguments(Mockito.any(FileInputStream.class)) + .thenReturn(gzipInputStreamMock); + } + + private void mockSequenceFileReader() throws Exception { + PowerMockito.mockStatic(FileSystem.class); + FileSystem fs = Mockito.mock(FileSystem.class); + Mockito.when(FileSystem.get(Mockito.any(Configuration.class))) + .thenReturn(fs); + + Path fsPath = new Path(PATH); + 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); + } +} diff --git a/src/test/java/com/pinterest/secor/io/FileWriterTest.java b/src/test/java/com/pinterest/secor/io/FileWriterTest.java new file mode 100644 index 000000000..0de64eac1 --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/FileWriterTest.java @@ -0,0 +1,162 @@ +/** + * 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.FileOutputStream; +import java.io.IOException; + +import junit.framework.TestCase; + +import org.apache.commons.configuration.PropertiesConfiguration; +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.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.GzipFileWriter; + +/** + * + * Test the file writers + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ FileSystem.class, SequenceFile.class, GzipFileWriter.class }) +public class FileWriterTest 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 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 setupSequenceFileWriterConfig() { + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.addProperty("secor.file.writer", + "com.pinterest.secor.io.impl.SequenceFileWriter"); + mConfig = new SecorConfig(properties); + } + + private void setupGzipFileWriterConfig() { + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.addProperty("secor.file.writer", + "com.pinterest.secor.io.impl.GzipFileWriter"); + mConfig = new SecorConfig(properties); + } + + public void testSequenceFileWriter() throws Exception { + setupSequenceFileWriterConfig(); + mockSequenceFileWriter(); + + FileWriter writer = FileWriterFactory.create(mLogFilePath, null, + mConfig); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem.get(Mockito.any(Configuration.class)); + + assert writer.getLength() == 123L; + + mockSequenceCompressedFileWriter(); + + FileWriterFactory.create(mLogFilePathGz, new GzipCodec(), mConfig); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem.get(Mockito.any(Configuration.class)); + + assert writer.getLength() == 12L; + } + + public void testGzipFileWriter() throws Exception { + setupGzipFileWriterConfig(); + mockGzipFileWriter(); + FileWriter writer = FileWriterFactory.create(mLogFilePathGz, null, + mConfig); + assert writer.getLength() == 0L; + } + + private void mockGzipFileWriter() throws Exception { + FileOutputStream fileOutputStreamMock = PowerMockito + .mock(FileOutputStream.class); + + PowerMockito.whenNew(FileOutputStream.class) + .withParameterTypes(String.class) + .withArguments(Mockito.any(String.class)) + .thenReturn(fileOutputStreamMock); + } + + private void mockSequenceFileWriter() throws IOException { + PowerMockito.mockStatic(FileSystem.class); + FileSystem fs = Mockito.mock(FileSystem.class); + Mockito.when(FileSystem.get(Mockito.any(Configuration.class))) + .thenReturn(fs); + + PowerMockito.mockStatic(SequenceFile.class); + Path fsPath = new Path(PATH); + SequenceFile.Writer writer = Mockito.mock(SequenceFile.Writer.class); + Mockito.when( + SequenceFile.createWriter(Mockito.eq(fs), + Mockito.any(Configuration.class), Mockito.eq(fsPath), + Mockito.eq(LongWritable.class), + Mockito.eq(BytesWritable.class))).thenReturn(writer); + + Mockito.when(writer.getLength()).thenReturn(123L); + } + + private void mockSequenceCompressedFileWriter() throws IOException { + 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); + + Mockito.when(writer.getLength()).thenReturn(12L); + } +} diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index cd96f8a3f..269cba6f0 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -17,13 +17,13 @@ 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.util.FileUtil; import com.pinterest.secor.util.IdUtil; + import junit.framework.TestCase; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.*; import org.junit.runner.RunWith; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -41,24 +41,23 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ @RunWith(PowerMockRunner.class) -@PrepareForTest({FileUtil.class, FileSystem.class, IdUtil.class}) +@PrepareForTest({FileUtil.class, IdUtil.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); + mReader = Mockito.mock(FileReader.class); } @Override - protected SequenceFile.Reader createReader(FileSystem fileSystem, Path path, - Configuration configuration) throws IOException { + protected FileReader createReader(LogFilePath srcPath) throws IOException { return mReader; } - public SequenceFile.Reader getReader() { + public FileReader getReader() { return mReader; } } @@ -149,30 +148,23 @@ public void testTrimFiles() throws Exception { logFilePaths.add(mLogFilePath); Mockito.when(mFileRegistry.getPaths(mTopicPartition)).thenReturn(logFilePaths); - PowerMockito.mockStatic(FileSystem.class); - - SequenceFile.Reader reader = mUploader.getReader(); - Mockito.doReturn(LongWritable.class).when(reader).getKeyClass(); - Mockito.doReturn(BytesWritable.class).when(reader).getValueClass(); + FileReader reader = mUploader.getReader(); - 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"); - SequenceFile.Writer writer = Mockito.mock(SequenceFile.Writer.class); + 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"); @@ -180,8 +172,8 @@ public Boolean answer(InvocationOnMock invocation) throws Throwable { mUploader.applyPolicy(); - Mockito.verify(writer).append(Mockito.any(LongWritable.class), - Mockito.any(BytesWritable.class)); + Mockito.verify(writer).write(Mockito.any(long.class), + Mockito.any(byte[].class)); Mockito.verify(mFileRegistry).deletePath(mLogFilePath); } } From 495413e7b5ac34b3a6ef5c536780548fe39b3896 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Thu, 25 Sep 2014 15:29:03 -0400 Subject: [PATCH 010/330] add performance test --- pom.xml | 12 + src/main/config/secor.common.properties | 3 + .../config/secor.test.perf.backup.properties | 18 ++ .../pinterest/secor/common/KafkaClient.java | 2 +- .../secor/common/LogFilePathAttributes.java | 62 +++++ .../pinterest/secor/common/SecorConfig.java | 8 + .../pinterest/secor/consumer/Consumer.java | 17 +- .../pinterest/secor/io/FileWriterFactory.java | 2 +- .../secor/io/impl/GzipFileReader.java | 6 +- .../secor/io/impl/GzipFileWriter.java | 10 +- .../secor/parser/PartitionFinalizer.java | 15 +- .../pinterest/secor/uploader/Uploader.java | 15 +- .../secor/performance/PerformanceTest.java | 261 ++++++++++++++++++ 13 files changed, 402 insertions(+), 29 deletions(-) create mode 100644 src/main/config/secor.test.perf.backup.properties create mode 100644 src/main/java/com/pinterest/secor/common/LogFilePathAttributes.java create mode 100644 src/test/java/com/pinterest/secor/performance/PerformanceTest.java diff --git a/pom.xml b/pom.xml index b08a4d752..4053c7af6 100644 --- a/pom.xml +++ b/pom.xml @@ -293,6 +293,18 @@ + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + test-jar + + + + diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 36e061fde..774d1677b 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -100,6 +100,9 @@ message.timestamp.input.pattern= # To enable compression, set this to a valid compression codec, such as 'org.apache.hadoop.io.compress.GzipCodec'. secor.compression.codec= +# If a codec is not specified you can provide an override extension +secor.file.extension= + # The secor file writer used to write the data, by default we write sequence files secor.file.writer=com.pinterest.secor.io.impl.SequenceFileWriter 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..57d272163 --- /dev/null +++ b/src/main/config/secor.test.perf.backup.properties @@ -0,0 +1,18 @@ +# 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 + +secor.kafka.perf_topic_prefix=PERF_PRAV \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 2f48e1123..b7edc61b1 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -136,7 +136,7 @@ private Message getMessage(TopicPartition topicPartition, long offset, messageAndOffset.offset(), payloadBytes); } - 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()); diff --git a/src/main/java/com/pinterest/secor/common/LogFilePathAttributes.java b/src/main/java/com/pinterest/secor/common/LogFilePathAttributes.java new file mode 100644 index 000000000..0c700f18f --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/LogFilePathAttributes.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import org.apache.hadoop.io.compress.CompressionCodec; + +import com.pinterest.secor.util.ReflectionUtil; + +/** + * Attributes for a log file path + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class LogFilePathAttributes { + + private final CompressionCodec mCodec; + private final String mFileExtension; + + public LogFilePathAttributes(SecorConfig mConfig) throws Exception { + if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + mCodec = + ((CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec())); + mFileExtension = mCodec.getDefaultExtension(); + } else { + mCodec = null; + String fileExtension = mConfig.getFileExtension(); + mFileExtension = (fileExtension == null)? "" : fileExtension; + } + } + + /** + * Get the log file path extension to be used + * @return + */ + public String getLogFileExtension() { + return mFileExtension; + } + + /** + * Get the log file path compression codec to use + * @return + */ + public CompressionCodec getCompressionCodec() { + return mCodec; + } + +} diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index b11e5d16f..e12c464ea 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -198,6 +198,10 @@ public String getMessageTimestampInputPattern() { public String getCompressionCodec() { return getString("secor.compression.codec"); } + + public String getFileExtension() { + return getString("secor.file.extension"); + } public int getMaxMessageSizeBytes() { return getInt("secor.max.message.size.bytes"); @@ -210,6 +214,10 @@ public String getFileWriter() { public String getFileReader() { return getString("secor.file.reader"); } + + public String getPerfTestTopicPrefix() { + return getString("secor.kafka.perf_topic_prefix"); + } private void checkProperty(String name) { if (!mProperties.containsKey(name)) { diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 68857d837..f615cd7ee 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -19,6 +19,7 @@ import com.pinterest.secor.common.FileRegistry; import com.pinterest.secor.common.OffsetTracker; import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.common.TopicPartition; import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; import com.pinterest.secor.parser.MessageParser; @@ -53,6 +54,7 @@ public class Consumer extends Thread { private MessageReader mMessageReader; private MessageWriter mMessageWriter; private MessageParser mMessageParser; + private OffsetTracker offsetTracker; private Uploader mUploader; // TODO(pawel): we should keep a count per topic partition. private double mUnparsableMessages; @@ -62,7 +64,7 @@ public Consumer(SecorConfig config) { } private void init() throws Exception { - OffsetTracker offsetTracker = new OffsetTracker(); + offsetTracker = new OffsetTracker(); mMessageReader = new MessageReader(mConfig, offsetTracker); FileRegistry fileRegistry = new FileRegistry(mConfig); mMessageWriter = new MessageWriter(mConfig, offsetTracker, fileRegistry); @@ -132,4 +134,17 @@ public void run() { } } } + + /** + * Helper to get the last offset for a topic partition (used in tests) + * @param topic + * @param partition + * @return + */ + public long getOffset(String topic, int partition) { + if (this.offsetTracker == null) + return -1; + return this.offsetTracker.getLastSeenOffset(new TopicPartition(topic, + partition)) + 1; + } } diff --git a/src/main/java/com/pinterest/secor/io/FileWriterFactory.java b/src/main/java/com/pinterest/secor/io/FileWriterFactory.java index cee5817d2..ff1799ca4 100644 --- a/src/main/java/com/pinterest/secor/io/FileWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/FileWriterFactory.java @@ -34,7 +34,7 @@ public class FileWriterFactory { public static FileWriter create(LogFilePath path, CompressionCodec codec, SecorConfig mConfig) throws Exception { if (mConfig.getFileWriter() != null && !mConfig.getFileWriter().isEmpty()) { return ((FileWriter) ReflectionUtil.createFileWriter(mConfig.getFileWriter(), path, codec)); - } + } throw new IllegalArgumentException("File Writer not defined or empty"); } diff --git a/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java b/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java index 7ff508dce..b024a7fb1 100644 --- a/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java +++ b/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java @@ -16,8 +16,10 @@ */ package com.pinterest.secor.io.impl; +import java.io.BufferedInputStream; import java.io.ByteArrayOutputStream; import java.io.EOFException; +import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.util.zip.GZIPInputStream; @@ -36,12 +38,12 @@ public class GzipFileReader implements FileReader { private static final byte DELIMITER = '\n'; - private final GZIPInputStream reader; + private final BufferedInputStream reader; private long offset; // constructor public GzipFileReader(LogFilePath path) throws Exception { - reader = new GZIPInputStream(new FileInputStream(path.getLogFilePath())); + reader = new BufferedInputStream(new GZIPInputStream(new FileInputStream(new File(path.getLogFilePath())))); this.offset = path.getOffset(); } diff --git a/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java b/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java index 18e9c37b9..dec41b998 100644 --- a/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java +++ b/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java @@ -16,6 +16,8 @@ */ package com.pinterest.secor.io.impl; +import java.io.BufferedOutputStream; +import java.io.File; import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; @@ -38,13 +40,15 @@ public class GzipFileWriter implements FileWriter { private static final byte DELIMITER = '\n'; private final CountingOutputStream countingStream; - private final GZIPOutputStream writer; + private final BufferedOutputStream writer; // constructor public GzipFileWriter(LogFilePath path) throws FileNotFoundException, IOException { - this.countingStream = new CountingOutputStream(new FileOutputStream(path.getLogFilePath())); - this.writer = new GZIPOutputStream(this.countingStream); + File logFile = new File(path.getLogFilePath()); + logFile.getParentFile().mkdirs(); + this.countingStream = new CountingOutputStream(new FileOutputStream(logFile)); + this.writer = new BufferedOutputStream(new GZIPOutputStream(this.countingStream)); } @Override diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index ec9d78f8a..84803572b 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -19,8 +19,6 @@ import com.pinterest.secor.common.*; import com.pinterest.secor.message.Message; 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; @@ -46,7 +44,7 @@ public class PartitionFinalizer { private ThriftMessageParser mThriftMessageParser; private KafkaClient mKafkaClient; private QuboleClient mQuboleClient; - private String mFileExtension; + private LogFilePathAttributes mFilePathAttributes; public PartitionFinalizer(SecorConfig config) throws Exception { mConfig = config; @@ -54,12 +52,7 @@ public PartitionFinalizer(SecorConfig config) throws Exception { mZookeeperConnector = new ZookeeperConnector(mConfig); mThriftMessageParser = new ThriftMessageParser(mConfig); mQuboleClient = new QuboleClient(mConfig); - if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { - CompressionCodec codec = (CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec()); - mFileExtension = codec.getDefaultExtension(); - } else { - mFileExtension = ""; - } + mFilePathAttributes = new LogFilePathAttributes(config); } private long getLastTimestampMillis(TopicPartition topicPartition) throws TException { @@ -117,7 +110,7 @@ private NavigableSet getPartitions(String topic) throws IOException, P final String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); String[] partitions = {"dt="}; LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, partitions, - mConfig.getGeneration(), 0, 0, mFileExtension); + mConfig.getGeneration(), 0, 0, mFilePathAttributes.getLogFileExtension()); String parentDir = logFilePath.getLogFileParentDir(); String[] partitionDirs = FileUtil.list(parentDir); Pattern pattern = Pattern.compile(".*/dt=(\\d\\d\\d\\d-\\d\\d-\\d\\d)$"); @@ -147,7 +140,7 @@ private void finalizePartitionsUpTo(String topic, Calendar calendar) throws IOEx String partitionStr = format.format(partition.getTime()); String[] partitions = {"dt=" + partitionStr}; LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, partitions, - mConfig.getGeneration(), 0, 0, mFileExtension); + mConfig.getGeneration(), 0, 0, mFilePathAttributes.getLogFileExtension()); String logFileDir = logFilePath.getLogFileDir(); assert FileUtil.exists(logFileDir) : "FileUtil.exists(" + logFileDir + ")"; String successFilePath = logFileDir + "/_SUCCESS"; diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index 36c94818e..c87afcbe2 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -23,9 +23,7 @@ import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.IdUtil; -import com.pinterest.secor.util.ReflectionUtil; -import org.apache.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -119,12 +117,7 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { mFileRegistry.deleteWriter(srcPath); try { reader = createReader(srcPath); - CompressionCodec codec = null; - String extension = ""; - if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { - codec = (CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec()); - extension = codec.getDefaultExtension(); - } + LogFilePathAttributes mFilePathAttributes = new LogFilePathAttributes(mConfig); KeyValue keyVal; while ((keyVal = reader.next()) != null) { if (keyVal.getKey() >= startOffset) { @@ -133,8 +126,10 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { IdUtil.getLocalMessageDir(); dstPath = new LogFilePath(localPrefix, srcPath.getTopic(), srcPath.getPartitions(), srcPath.getGeneration(), - srcPath.getKafkaPartition(), startOffset, extension); - writer = mFileRegistry.getOrCreateWriter(dstPath, codec); + srcPath.getKafkaPartition(), startOffset, + mFilePathAttributes.getLogFileExtension()); + writer = mFileRegistry.getOrCreateWriter(dstPath, + mFilePathAttributes.getCompressionCodec()); } writer.write(keyVal.getKey(), keyVal.getValue()); copiedMessages++; diff --git a/src/test/java/com/pinterest/secor/performance/PerformanceTest.java b/src/test/java/com/pinterest/secor/performance/PerformanceTest.java new file mode 100644 index 000000000..1f7dbcc56 --- /dev/null +++ b/src/test/java/com/pinterest/secor/performance/PerformanceTest.java @@ -0,0 +1,261 @@ +/** + * 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.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.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 PerformanceTest { + + public static void main(String[] args) throws Exception { + if (args.length != 4) { + System.err.println("USAGE: java " + PerformanceTest.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++) { + long val = consumer.getOffset(topic, i); + 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); + 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 From ffd7e6ac454276d9d3843be90d00181eb44f23a7 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Fri, 26 Sep 2014 15:36:02 -0400 Subject: [PATCH 011/330] switch to hadoop Gzip (faster) and fix merge bug --- src/main/config/secor.test.perf.backup.properties | 3 ++- .../java/com/pinterest/secor/io/impl/GzipFileReader.java | 8 ++++++-- .../java/com/pinterest/secor/io/impl/GzipFileWriter.java | 8 ++++++-- .../java/com/pinterest/secor/writer/MessageWriter.java | 1 - .../com/pinterest/secor/performance/PerformanceTest.java | 2 ++ 5 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/main/config/secor.test.perf.backup.properties b/src/main/config/secor.test.perf.backup.properties index 57d272163..3fd77f9b2 100644 --- a/src/main/config/secor.test.perf.backup.properties +++ b/src/main/config/secor.test.perf.backup.properties @@ -15,4 +15,5 @@ include=secor.dev.backup.properties -secor.kafka.perf_topic_prefix=PERF_PRAV \ No newline at end of file +#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/java/com/pinterest/secor/io/impl/GzipFileReader.java b/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java index b024a7fb1..308408e7b 100644 --- a/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java +++ b/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java @@ -22,7 +22,9 @@ import java.io.File; import java.io.FileInputStream; import java.io.IOException; -import java.util.zip.GZIPInputStream; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.compress.GzipCodec; import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.io.FileReader; @@ -43,7 +45,9 @@ public class GzipFileReader implements FileReader { // constructor public GzipFileReader(LogFilePath path) throws Exception { - reader = new BufferedInputStream(new GZIPInputStream(new FileInputStream(new File(path.getLogFilePath())))); + GzipCodec codec = new GzipCodec(); + codec.setConf(new Configuration()); + reader = new BufferedInputStream(codec.createInputStream(new FileInputStream(new File(path.getLogFilePath())))); this.offset = path.getOffset(); } diff --git a/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java b/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java index dec41b998..8b9d88a7c 100644 --- a/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java +++ b/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java @@ -21,7 +21,9 @@ import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; -import java.util.zip.GZIPOutputStream; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.compress.GzipCodec; import com.google.common.io.CountingOutputStream; import com.pinterest.secor.common.LogFilePath; @@ -45,10 +47,12 @@ public class GzipFileWriter implements FileWriter { // constructor public GzipFileWriter(LogFilePath path) throws FileNotFoundException, IOException { + GzipCodec codec = new GzipCodec(); + codec.setConf(new Configuration()); File logFile = new File(path.getLogFilePath()); logFile.getParentFile().mkdirs(); this.countingStream = new CountingOutputStream(new FileOutputStream(logFile)); - this.writer = new BufferedOutputStream(new GZIPOutputStream(this.countingStream)); + this.writer = new BufferedOutputStream(codec.createOutputStream((this.countingStream))); } @Override diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index 7c582d57f..bfb68e0e3 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -67,7 +67,6 @@ public void adjustOffset(Message message) throws IOException { } public void write(ParsedMessage message) throws Exception { - adjustOffset(message); TopicPartition topicPartition = new TopicPartition(message.getTopic(), message.getKafkaPartition()); long offset = mOffsetTracker.getAdjustedCommittedOffsetCount(topicPartition); diff --git a/src/test/java/com/pinterest/secor/performance/PerformanceTest.java b/src/test/java/com/pinterest/secor/performance/PerformanceTest.java index 1f7dbcc56..c56fece72 100644 --- a/src/test/java/com/pinterest/secor/performance/PerformanceTest.java +++ b/src/test/java/com/pinterest/secor/performance/PerformanceTest.java @@ -170,6 +170,8 @@ public void uncaughtException(Thread thread, Throwable exception) { 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; } } From 6c8cd1d13d688152a3cb76f56062faa9a6d04095 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Mon, 6 Oct 2014 20:21:15 -0700 Subject: [PATCH 012/330] incorporate all changes suggested by pawel --- src/main/config/secor.common.properties | 10 +- .../pinterest/secor/common/FileRegistry.java | 19 +- .../secor/common/LogFilePathAttributes.java | 62 --- .../pinterest/secor/common/SecorConfig.java | 8 +- .../pinterest/secor/consumer/Consumer.java | 26 +- .../pinterest/secor/io/FileReaderFactory.java | 45 --- ...{FileReader.java => FileReaderWriter.java} | 47 ++- .../com/pinterest/secor/io/FileWriter.java | 51 --- .../pinterest/secor/io/FileWriterFactory.java | 41 -- .../impl/DelimitedTextFileReaderWriter.java | 111 ++++++ .../secor/io/impl/GzipFileReader.java | 76 ---- .../secor/io/impl/GzipFileWriter.java | 74 ---- .../secor/io/impl/SequenceFileReader.java | 70 ---- .../io/impl/SequenceFileReaderWriter.java | 97 +++++ .../secor/io/impl/SequenceFileWriter.java | 76 ---- .../secor/parser/PartitionFinalizer.java | 16 +- .../pinterest/secor/uploader/Uploader.java | 30 +- .../pinterest/secor/util/ReflectionUtil.java | 53 +-- .../pinterest/secor/writer/MessageWriter.java | 20 +- .../secor/common/FileRegistryTest.java | 83 ++-- .../pinterest/secor/io/FileReaderTest.java | 134 ------- .../secor/io/FileReaderWriterTest.java | 205 ++++++++++ .../pinterest/secor/io/FileWriterTest.java | 162 -------- .../secor/performance/PerformanceTest.java | 365 +++++++++--------- .../secor/uploader/UploaderTest.java | 112 ++++-- 25 files changed, 840 insertions(+), 1153 deletions(-) delete mode 100644 src/main/java/com/pinterest/secor/common/LogFilePathAttributes.java delete mode 100644 src/main/java/com/pinterest/secor/io/FileReaderFactory.java rename src/main/java/com/pinterest/secor/io/{FileReader.java => FileReaderWriter.java} (55%) delete mode 100644 src/main/java/com/pinterest/secor/io/FileWriter.java delete mode 100644 src/main/java/com/pinterest/secor/io/FileWriterFactory.java create mode 100644 src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java delete mode 100644 src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java delete mode 100644 src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java delete mode 100644 src/main/java/com/pinterest/secor/io/impl/SequenceFileReader.java create mode 100644 src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java delete mode 100644 src/main/java/com/pinterest/secor/io/impl/SequenceFileWriter.java delete mode 100644 src/test/java/com/pinterest/secor/io/FileReaderTest.java create mode 100644 src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java delete mode 100644 src/test/java/com/pinterest/secor/io/FileWriterTest.java diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 774d1677b..26d82890e 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -100,14 +100,8 @@ message.timestamp.input.pattern= # To enable compression, set this to a valid compression codec, such as 'org.apache.hadoop.io.compress.GzipCodec'. secor.compression.codec= -# If a codec is not specified you can provide an override extension -secor.file.extension= - -# The secor file writer used to write the data, by default we write sequence files -secor.file.writer=com.pinterest.secor.io.impl.SequenceFileWriter - -#The secor file reader used to read the data, by default we write/read sequence files -secor.file.reader=com.pinterest.secor.io.impl.SequenceFileReader +# The secor file reader/writer used to read/write the data, by default we write sequence files +secor.file.reader.writer=com.pinterest.secor.io.impl.SequenceFileReaderWriter # 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 diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index f776e9ec2..64bb9e45c 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -16,9 +16,9 @@ */ package com.pinterest.secor.common; -import com.pinterest.secor.io.FileWriter; -import com.pinterest.secor.io.FileWriterFactory; +import com.pinterest.secor.io.FileReaderWriter; import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.util.ReflectionUtil; import com.pinterest.secor.util.StatsUtil; import org.apache.hadoop.io.compress.CompressionCodec; @@ -39,13 +39,13 @@ public class FileRegistry { private final SecorConfig mConfig; private HashMap> mFiles; - private HashMap mWriters; + private HashMap mWriters; private HashMap mCreationTimes; public FileRegistry(SecorConfig mConfig) { this.mConfig = mConfig; mFiles = new HashMap>(); - mWriters = new HashMap(); + mWriters = new HashMap(); mCreationTimes = new HashMap(); } @@ -82,8 +82,9 @@ public Collection getPaths(TopicPartition topicPartition) { * @return Writer for a given path. * @throws Exception */ - public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) throws Exception { - FileWriter writer = mWriters.get(path); + public FileReaderWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) + throws Exception { + FileReaderWriter writer = mWriters.get(path); if (writer == null) { // Just in case. FileUtil.delete(path.getLogFilePath()); @@ -98,7 +99,7 @@ public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) th if (!files.contains(path)) { files.add(path); } - writer = FileWriterFactory.create(path, codec, this.mConfig); + writer = ((FileReaderWriter) ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), path, codec)); mWriters.put(path, writer); mCreationTimes.put(path, System.currentTimeMillis() / 1000L); LOG.debug("created writer for path " + path.getLogFilePath()); @@ -149,7 +150,7 @@ public void deleteTopicPartition(TopicPartition topicPartition) throws IOExcepti * @param path The path to remove the writer for. */ public void deleteWriter(LogFilePath path) throws IOException { - FileWriter writer = mWriters.get(path); + FileReaderWriter writer = mWriters.get(path); if (writer == null) { LOG.warn("No writer found for path " + path.getLogFilePath()); } else { @@ -187,7 +188,7 @@ public long getSize(TopicPartition topicPartition) throws IOException { Collection paths = getPaths(topicPartition); long result = 0; for (LogFilePath path : paths) { - FileWriter writer = mWriters.get(path); + FileReaderWriter writer = mWriters.get(path); if (writer != null) { result += writer.getLength(); } diff --git a/src/main/java/com/pinterest/secor/common/LogFilePathAttributes.java b/src/main/java/com/pinterest/secor/common/LogFilePathAttributes.java deleted file mode 100644 index 0c700f18f..000000000 --- a/src/main/java/com/pinterest/secor/common/LogFilePathAttributes.java +++ /dev/null @@ -1,62 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.pinterest.secor.common; - -import org.apache.hadoop.io.compress.CompressionCodec; - -import com.pinterest.secor.util.ReflectionUtil; - -/** - * Attributes for a log file path - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -public class LogFilePathAttributes { - - private final CompressionCodec mCodec; - private final String mFileExtension; - - public LogFilePathAttributes(SecorConfig mConfig) throws Exception { - if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { - mCodec = - ((CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec())); - mFileExtension = mCodec.getDefaultExtension(); - } else { - mCodec = null; - String fileExtension = mConfig.getFileExtension(); - mFileExtension = (fileExtension == null)? "" : fileExtension; - } - } - - /** - * Get the log file path extension to be used - * @return - */ - public String getLogFileExtension() { - return mFileExtension; - } - - /** - * Get the log file path compression codec to use - * @return - */ - public CompressionCodec getCompressionCodec() { - return mCodec; - } - -} diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index e12c464ea..01926fe9b 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -207,12 +207,8 @@ public int getMaxMessageSizeBytes() { return getInt("secor.max.message.size.bytes"); } - public String getFileWriter() { - return getString("secor.file.writer"); - } - - public String getFileReader() { - return getString("secor.file.reader"); + public String getFileReaderWriter() { + return getString("secor.file.reader.writer"); } public String getPerfTestTopicPrefix() { diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index f615cd7ee..f5cc04d24 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -19,7 +19,6 @@ import com.pinterest.secor.common.FileRegistry; import com.pinterest.secor.common.OffsetTracker; import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.common.TopicPartition; import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; import com.pinterest.secor.parser.MessageParser; @@ -54,7 +53,7 @@ public class Consumer extends Thread { private MessageReader mMessageReader; private MessageWriter mMessageWriter; private MessageParser mMessageParser; - private OffsetTracker offsetTracker; + private OffsetTracker mOffsetTracker; private Uploader mUploader; // TODO(pawel): we should keep a count per topic partition. private double mUnparsableMessages; @@ -64,13 +63,13 @@ public Consumer(SecorConfig config) { } private void init() throws Exception { - offsetTracker = new OffsetTracker(); - mMessageReader = new MessageReader(mConfig, offsetTracker); + mOffsetTracker = new OffsetTracker(); + mMessageReader = new MessageReader(mConfig, mOffsetTracker); FileRegistry fileRegistry = new FileRegistry(mConfig); - mMessageWriter = new MessageWriter(mConfig, offsetTracker, fileRegistry); + mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, fileRegistry); mMessageParser = (MessageParser) ReflectionUtil.createMessageParser( mConfig.getMessageParserClass(), mConfig); - mUploader = new Uploader(mConfig, offsetTracker, fileRegistry); + mUploader = new Uploader(mConfig, mOffsetTracker, fileRegistry); mUnparsableMessages = 0.; } @@ -134,17 +133,16 @@ public void run() { } } } - + /** - * Helper to get the last offset for a topic partition (used in tests) + * Helper to get the offset tracker (used in tests) + * * @param topic * @param partition * @return */ - public long getOffset(String topic, int partition) { - if (this.offsetTracker == null) - return -1; - return this.offsetTracker.getLastSeenOffset(new TopicPartition(topic, - partition)) + 1; - } + public OffsetTracker getOffsetTracker() { + return this.mOffsetTracker; + } + } diff --git a/src/main/java/com/pinterest/secor/io/FileReaderFactory.java b/src/main/java/com/pinterest/secor/io/FileReaderFactory.java deleted file mode 100644 index 0e71cf8ab..000000000 --- a/src/main/java/com/pinterest/secor/io/FileReaderFactory.java +++ /dev/null @@ -1,45 +0,0 @@ -/** - * 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 com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.util.ReflectionUtil; - -/** - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -public class FileReaderFactory { - - /** - * Given a log file path create a file reader using the file reader configured - * - * @param path - * @param mConfig - * @return - * @throws Exception - */ - public static FileReader create(LogFilePath path, SecorConfig mConfig) throws Exception { - if (mConfig.getFileReader() != null && !mConfig.getFileReader().isEmpty()) { - return ((FileReader) ReflectionUtil.createFileReader(mConfig.getFileReader(), path)); - } - throw new IllegalArgumentException("File Reader not defined or empty"); - } - -} diff --git a/src/main/java/com/pinterest/secor/io/FileReader.java b/src/main/java/com/pinterest/secor/io/FileReaderWriter.java similarity index 55% rename from src/main/java/com/pinterest/secor/io/FileReader.java rename to src/main/java/com/pinterest/secor/io/FileReaderWriter.java index f96b8e213..15b105ed8 100644 --- a/src/main/java/com/pinterest/secor/io/FileReader.java +++ b/src/main/java/com/pinterest/secor/io/FileReaderWriter.java @@ -19,24 +19,43 @@ import java.io.IOException; /** - * Generic file reader interface for all secor files + * Generic file reader/writer interface for all secor files * * @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; +public interface FileReaderWriter { - /** - * Close the file - * @throws IOException - */ - public void close() throws IOException; + /** + * 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; + + /** + * Get length of data written up to now to the underlying file + * + * @return + * @throws IOException + */ + public long getLength() throws IOException; + + /** + * Write the given key and value to the file + * + * @param key + * @param value + * @throws IOException + */ + public void write(long key, byte[] value) throws IOException; } diff --git a/src/main/java/com/pinterest/secor/io/FileWriter.java b/src/main/java/com/pinterest/secor/io/FileWriter.java deleted file mode 100644 index 0df0ea7eb..000000000 --- a/src/main/java/com/pinterest/secor/io/FileWriter.java +++ /dev/null @@ -1,51 +0,0 @@ -/** - * 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 all secor files - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -public interface FileWriter { - - /** - * Close the file - * @throws IOException - */ - public void close() throws IOException; - - /** - * Get length of data written up to now to the underlying file - * @return - * @throws IOException - */ - public long getLength() throws IOException; - - /** - * Write the given key and value to the file - * - * @param key - * @param value - * @throws IOException - */ - public void write(long key, byte[] value) throws IOException; - -} diff --git a/src/main/java/com/pinterest/secor/io/FileWriterFactory.java b/src/main/java/com/pinterest/secor/io/FileWriterFactory.java deleted file mode 100644 index ff1799ca4..000000000 --- a/src/main/java/com/pinterest/secor/io/FileWriterFactory.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * 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 org.apache.hadoop.io.compress.CompressionCodec; - -import com.pinterest.secor.common.LogFilePath; -import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.util.ReflectionUtil; - -/** - * File Writer Factory which returns a writer based upon secor configuration - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -public class FileWriterFactory { - - public static FileWriter create(LogFilePath path, CompressionCodec codec, SecorConfig mConfig) throws Exception { - if (mConfig.getFileWriter() != null && !mConfig.getFileWriter().isEmpty()) { - return ((FileWriter) ReflectionUtil.createFileWriter(mConfig.getFileWriter(), path, codec)); - } - throw new IllegalArgumentException("File Writer not defined or empty"); - } - -} diff --git a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java new file mode 100644 index 000000000..9c11cf5fc --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java @@ -0,0 +1,111 @@ +/** + * 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.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import org.apache.hadoop.io.compress.CompressionCodec; + +import com.google.common.io.CountingOutputStream; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.FileReaderWriter; +import com.pinterest.secor.io.KeyValue; + +/** + * + * Delimited Text File Reader Writer with Compression + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class DelimitedTextFileReaderWriter implements FileReaderWriter { + + // delimiter used between messages + private static final byte DELIMITER = '\n'; + + private final CountingOutputStream countingStream; + private final BufferedOutputStream writer; + + private final BufferedInputStream reader; + private long offset; + + // constructor + public DelimitedTextFileReaderWriter(LogFilePath path, + CompressionCodec codec) throws FileNotFoundException, IOException { + File logFile = new File(path.getLogFilePath()); + logFile.getParentFile().mkdirs(); + this.countingStream = new CountingOutputStream(new FileOutputStream( + logFile)); + this.writer = (codec == null) ? new BufferedOutputStream( + this.countingStream) : new BufferedOutputStream( + codec.createOutputStream(this.countingStream)); + + InputStream inputStream = new FileInputStream(new File( + path.getLogFilePath())); + this.reader = (codec == null) ? new BufferedInputStream(inputStream) + : new BufferedInputStream(codec.createInputStream(inputStream)); + this.offset = path.getOffset(); + } + + @Override + public void close() throws IOException { + if (this.writer != null) { + this.writer.close(); + } + if (this.reader != null) { + this.reader.close(); + } + } + + @Override + public long getLength() throws IOException { + return this.countingStream.getCount(); + } + + @Override + public void write(long key, byte[] value) throws IOException { + this.writer.write(value); + this.writer.write(DELIMITER); + } + + @Override + public KeyValue next() throws IOException { + ByteArrayOutputStream messageBuffer = new ByteArrayOutputStream(); + int nextByte; + while ((nextByte = reader.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.offset++, messageBuffer.toByteArray()); + } + +} diff --git a/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java b/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java deleted file mode 100644 index 308408e7b..000000000 --- a/src/main/java/com/pinterest/secor/io/impl/GzipFileReader.java +++ /dev/null @@ -1,76 +0,0 @@ -/** - * 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.ByteArrayOutputStream; -import java.io.EOFException; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.compress.GzipCodec; - -import com.pinterest.secor.common.LogFilePath; -import com.pinterest.secor.io.FileReader; -import com.pinterest.secor.io.KeyValue; - -/** - * Gzip File reader implementation - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -public class GzipFileReader implements FileReader { - - private static final byte DELIMITER = '\n'; - - private final BufferedInputStream reader; - private long offset; - - // constructor - public GzipFileReader(LogFilePath path) throws Exception { - GzipCodec codec = new GzipCodec(); - codec.setConf(new Configuration()); - reader = new BufferedInputStream(codec.createInputStream(new FileInputStream(new File(path.getLogFilePath())))); - this.offset = path.getOffset(); - } - - @Override - public KeyValue next() throws IOException { - ByteArrayOutputStream messageBuffer = new ByteArrayOutputStream(); - int nextByte; - while ((nextByte = reader.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.offset++, messageBuffer.toByteArray()); - } - - @Override - public void close() throws IOException { - this.reader.close(); - } - -} diff --git a/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java b/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java deleted file mode 100644 index 8b9d88a7c..000000000 --- a/src/main/java/com/pinterest/secor/io/impl/GzipFileWriter.java +++ /dev/null @@ -1,74 +0,0 @@ -/** - * 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.BufferedOutputStream; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.compress.GzipCodec; - -import com.google.common.io.CountingOutputStream; -import com.pinterest.secor.common.LogFilePath; -import com.pinterest.secor.io.FileWriter; - -/** - * - * Gzip File writer implementation - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -public class GzipFileWriter implements FileWriter { - - //delimiter used between messages - private static final byte DELIMITER = '\n'; - - private final CountingOutputStream countingStream; - private final BufferedOutputStream writer; - - // constructor - public GzipFileWriter(LogFilePath path) throws FileNotFoundException, - IOException { - GzipCodec codec = new GzipCodec(); - codec.setConf(new Configuration()); - File logFile = new File(path.getLogFilePath()); - logFile.getParentFile().mkdirs(); - this.countingStream = new CountingOutputStream(new FileOutputStream(logFile)); - this.writer = new BufferedOutputStream(codec.createOutputStream((this.countingStream))); - } - - @Override - public void close() throws IOException { - this.writer.close(); - } - - @Override - public long getLength() throws IOException { - return this.countingStream.getCount(); - } - - @Override - public void write(long key, byte[] value) throws IOException { - this.writer.write(value); - this.writer.write(DELIMITER); - } - -} diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReader.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReader.java deleted file mode 100644 index afe40a9cc..000000000 --- a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReader.java +++ /dev/null @@ -1,70 +0,0 @@ -/** - * 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 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.common.LogFilePath; -import com.pinterest.secor.io.FileReader; -import com.pinterest.secor.io.KeyValue; - -/** - * - * Sequence file reader implementation - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -public class SequenceFileReader implements FileReader { - - private final SequenceFile.Reader reader; - private final LongWritable key; - private final BytesWritable value; - - // constructor - public SequenceFileReader(LogFilePath path) throws Exception { - Configuration config = new Configuration(); - FileSystem fs = FileSystem.get(config); - String srcFilename = path.getLogFilePath(); - Path srcFsPath = new Path(srcFilename); - this.reader = new SequenceFile.Reader(fs, srcFsPath, config); - this.key = (LongWritable) reader.getKeyClass().newInstance(); - this.value = (BytesWritable) reader.getValueClass().newInstance(); - } - - @Override - public KeyValue next() throws IOException { - if(reader.next(key, value)) { - return new KeyValue(key.get(), value.getBytes()); - } else { - return null; - } - } - - @Override - public void close() throws IOException { - this.reader.close(); - } - -} diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java new file mode 100644 index 000000000..974232220 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.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.io.impl; + +import java.io.IOException; + +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 com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.FileReaderWriter; +import com.pinterest.secor.io.KeyValue; + +/** + * + * Sequence file writer implementation + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class SequenceFileReaderWriter implements FileReaderWriter { + + private final SequenceFile.Writer writer; + private final SequenceFile.Reader reader; + private final LongWritable key; + private final BytesWritable value; + + // constructor + public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec) + throws Exception { + Configuration config = new Configuration(); + FileSystem fs = FileSystem.get(config); + Path fsPath = new Path(path.getLogFilePath()); + if (codec != null) { + this.writer = SequenceFile.createWriter(fs, config, fsPath, + LongWritable.class, BytesWritable.class, + SequenceFile.CompressionType.BLOCK, codec); + } else { + this.writer = SequenceFile.createWriter(fs, config, fsPath, + LongWritable.class, BytesWritable.class); + } + this.reader = new SequenceFile.Reader(fs, fsPath, config); + this.key = (LongWritable) reader.getKeyClass().newInstance(); + this.value = (BytesWritable) reader.getValueClass().newInstance(); + } + + @Override + public void close() throws IOException { + if (this.writer == null) { + this.writer.close(); + } + if (this.reader == null) { + this.reader.close(); + } + } + + @Override + public long getLength() throws IOException { + return this.writer.getLength(); + } + + @Override + public void write(long key, byte[] value) throws IOException { + LongWritable writeableKey = new LongWritable(key); + BytesWritable writeableValue = new BytesWritable(value); + this.writer.append(writeableKey, writeableValue); + } + + @Override + public KeyValue next() throws IOException { + if (reader.next(key, value)) { + return new KeyValue(key.get(), value.getBytes()); + } else { + return null; + } + } + +} diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileWriter.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileWriter.java deleted file mode 100644 index cb7c5cf96..000000000 --- a/src/main/java/com/pinterest/secor/io/impl/SequenceFileWriter.java +++ /dev/null @@ -1,76 +0,0 @@ -/** - * 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 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 com.pinterest.secor.common.LogFilePath; -import com.pinterest.secor.io.FileWriter; - -/** - * - * Sequence file writer implementation - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -public class SequenceFileWriter implements FileWriter { - - private final SequenceFile.Writer writer; - - // constructor - public SequenceFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { - Configuration config = new Configuration(); - FileSystem fs = FileSystem.get(config); - if (codec != null) { - Path fsPath = new Path(path.getLogFilePath()); - this.writer = SequenceFile.createWriter(fs, config, fsPath, LongWritable.class, - BytesWritable.class, - SequenceFile.CompressionType.BLOCK, codec); - } else { - Path fsPath = new Path(path.getLogFilePath()); - this.writer = SequenceFile.createWriter(fs, config, fsPath, LongWritable.class, - BytesWritable.class); - } - } - - @Override - public void close() throws IOException { - this.writer.close(); - } - - @Override - public long getLength() throws IOException { - return this.writer.getLength(); - } - - @Override - public void write(long key, byte[] value) throws IOException { - LongWritable writeableKey = new LongWritable(key); - BytesWritable writeableValue = new BytesWritable(value); - this.writer.append(writeableKey, writeableValue); - } - -} diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 84803572b..9f03e18da 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -19,6 +19,9 @@ import com.pinterest.secor.common.*; import com.pinterest.secor.message.Message; 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; @@ -44,7 +47,7 @@ public class PartitionFinalizer { private ThriftMessageParser mThriftMessageParser; private KafkaClient mKafkaClient; private QuboleClient mQuboleClient; - private LogFilePathAttributes mFilePathAttributes; + private String mFileExtension; public PartitionFinalizer(SecorConfig config) throws Exception { mConfig = config; @@ -52,7 +55,12 @@ public PartitionFinalizer(SecorConfig config) throws Exception { mZookeeperConnector = new ZookeeperConnector(mConfig); mThriftMessageParser = new ThriftMessageParser(mConfig); mQuboleClient = new QuboleClient(mConfig); - mFilePathAttributes = new LogFilePathAttributes(config); + if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + CompressionCodec codec = ((CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec())); + mFileExtension = codec.getDefaultExtension(); + } else { + mFileExtension = ""; + } } private long getLastTimestampMillis(TopicPartition topicPartition) throws TException { @@ -110,7 +118,7 @@ private NavigableSet getPartitions(String topic) throws IOException, P final String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); String[] partitions = {"dt="}; LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, partitions, - mConfig.getGeneration(), 0, 0, mFilePathAttributes.getLogFileExtension()); + 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)$"); @@ -140,7 +148,7 @@ private void finalizePartitionsUpTo(String topic, Calendar calendar) throws IOEx String partitionStr = format.format(partition.getTime()); String[] partitions = {"dt=" + partitionStr}; LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, partitions, - mConfig.getGeneration(), 0, 0, mFilePathAttributes.getLogFileExtension()); + mConfig.getGeneration(), 0, 0, mFileExtension); String logFileDir = logFilePath.getLogFileDir(); assert FileUtil.exists(logFileDir) : "FileUtil.exists(" + logFileDir + ")"; String successFilePath = logFileDir + "/_SUCCESS"; diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index c87afcbe2..a5c3e4822 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -17,13 +17,13 @@ package com.pinterest.secor.uploader; import com.pinterest.secor.common.*; -import com.pinterest.secor.io.FileReader; -import com.pinterest.secor.io.FileReaderFactory; -import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.FileReaderWriter; import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.IdUtil; +import com.pinterest.secor.util.ReflectionUtil; +import org.apache.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -101,23 +101,31 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { * This method is intended to be overwritten in tests. * @throws Exception */ - protected FileReader createReader(LogFilePath srcPath) throws Exception { - return FileReaderFactory.create(srcPath, this.mConfig); + protected FileReaderWriter createReader(LogFilePath srcPath, CompressionCodec codec) throws Exception { + return (FileReaderWriter) ReflectionUtil.createFileReaderWriter( + mConfig.getFileReaderWriter(), + srcPath, + codec); } private void trim(LogFilePath srcPath, long startOffset) throws Exception { if (startOffset == srcPath.getOffset()) { return; } - FileReader reader = null; - FileWriter writer = null; + FileReaderWriter reader = null; + FileReaderWriter 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(srcPath); - LogFilePathAttributes mFilePathAttributes = new LogFilePathAttributes(mConfig); + CompressionCodec codec = null; + String extension = ""; + if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + codec = ((CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec())); + extension = codec.getDefaultExtension(); + } + reader = createReader(srcPath, codec); KeyValue keyVal; while ((keyVal = reader.next()) != null) { if (keyVal.getKey() >= startOffset) { @@ -127,9 +135,9 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { dstPath = new LogFilePath(localPrefix, srcPath.getTopic(), srcPath.getPartitions(), srcPath.getGeneration(), srcPath.getKafkaPartition(), startOffset, - mFilePathAttributes.getLogFileExtension()); + extension); writer = mFileRegistry.getOrCreateWriter(dstPath, - mFilePathAttributes.getCompressionCodec()); + codec); } writer.write(keyVal.getKey(), keyVal.getValue()); copiedMessages++; diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index bde40df0b..09fb7fc3e 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -53,41 +53,20 @@ public static Object createMessageParser(String className, throw new IllegalArgumentException("Class not found " + className); } - public static Object createFileWriter(String className, - LogFilePath logFilePath, CompressionCodec compressionCodec) - throws Exception { - Class clazz = Class.forName(className); - // Search for an "appropriate" constructor. - for (Constructor ctor : clazz.getConstructors()) { - Class[] paramTypes = ctor.getParameterTypes(); - - // If the arity matches, let's use it. - if (paramTypes.length == 1) { - Object[] args = { logFilePath }; - return ctor.newInstance(args); - } else if (paramTypes.length == 2) { - Object[] args = { logFilePath, compressionCodec }; - return ctor.newInstance(args); - } - } - throw new IllegalArgumentException("Class not found " + className); - - } + public static Object createFileReaderWriter(String className, + LogFilePath logFilePath, CompressionCodec compressionCodec) + throws Exception { + Class clazz = Class.forName(className); + // Search for an "appropriate" constructor. + for (Constructor ctor : clazz.getConstructors()) { + Class[] paramTypes = ctor.getParameterTypes(); - public static Object createFileReader(String className, LogFilePath path) - throws Exception { - Class clazz = Class.forName(className); - - // Search for an "appropriate" constructor. - for (Constructor ctor : clazz.getConstructors()) { - Class[] paramTypes = ctor.getParameterTypes(); - - // If the arity matches, let's use it. - if (paramTypes.length == 1) { - Object[] args = { path }; - return ctor.newInstance(args); - } - } - throw new IllegalArgumentException("Class not found " + className); - } -} + // If the arity matches, let's use it. + if (paramTypes.length == 2) { + Object[] args = { logFilePath, compressionCodec }; + return ctor.newInstance(args); + } + } + throw new IllegalArgumentException("Class not found " + className); + } +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index bfb68e0e3..2af1b1ff0 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -17,7 +17,7 @@ package com.pinterest.secor.writer; import com.pinterest.secor.common.*; -import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.FileReaderWriter; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; @@ -25,6 +25,9 @@ import java.io.IOException; import com.pinterest.secor.util.IdUtil; +import com.pinterest.secor.util.ReflectionUtil; + +import org.apache.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,7 +42,8 @@ public class MessageWriter { private SecorConfig mConfig; private OffsetTracker mOffsetTracker; private FileRegistry mFileRegistry; - private LogFilePathAttributes mFilePathAttributes; + private String mFileExtension; + private CompressionCodec mCodec; private String mLocalPrefix; public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, @@ -47,7 +51,12 @@ public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, mConfig = config; mOffsetTracker = offsetTracker; mFileRegistry = fileRegistry; - mFilePathAttributes = new LogFilePathAttributes(config); + if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + mCodec = ((CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec())); + mFileExtension = mCodec.getDefaultExtension(); + } else { + mFileExtension = ""; + } mLocalPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); } @@ -71,9 +80,8 @@ public void write(ParsedMessage message) throws Exception { message.getKafkaPartition()); long offset = mOffsetTracker.getAdjustedCommittedOffsetCount(topicPartition); LogFilePath path = new LogFilePath(mLocalPrefix, mConfig.getGeneration(), offset, message, - mFilePathAttributes.getLogFileExtension()); - FileWriter writer; - writer = mFileRegistry.getOrCreateWriter(path, mFilePathAttributes.getCompressionCodec()); + mFileExtension); + FileReaderWriter writer = mFileRegistry.getOrCreateWriter(path, mCodec); writer.write(message.getOffset(), message.getPayload()); LOG.debug("appended message " + message + " to file " + path.getLogFilePath() + ". File length " + writer.getLength()); diff --git a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java index 1f893ed92..08eef78a7 100644 --- a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java +++ b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java @@ -16,9 +16,9 @@ */ package com.pinterest.secor.common; -import com.pinterest.secor.io.FileWriter; -import com.pinterest.secor.io.FileWriterFactory; +import com.pinterest.secor.io.FileReaderWriter; import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.util.ReflectionUtil; import junit.framework.TestCase; @@ -39,17 +39,14 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ @RunWith(PowerMockRunner.class) -@PrepareForTest({FileRegistry.class, FileUtil.class, FileWriterFactory.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,7 +55,8 @@ public class FileRegistryTest extends TestCase { public void setUp() throws Exception { super.setUp(); PropertiesConfiguration properties = new PropertiesConfiguration(); - properties.addProperty("secor.file.writer", "com.pinterest.secor.io.impl.SequenceFileWriter"); + properties.addProperty("secor.file.reader.writer", + "com.pinterest.secor.io.impl.SequenceFileWriter"); SecorConfig secorConfig = new SecorConfig(properties); mRegistry = new FileRegistry(secorConfig); mLogFilePath = new LogFilePath("/some_parent_dir", PATH); @@ -69,28 +67,33 @@ public void setUp() throws Exception { private void createWriter() throws Exception { PowerMockito.mockStatic(FileUtil.class); - PowerMockito.mockStatic(FileWriterFactory.class); - FileWriter writer = Mockito.mock(FileWriter.class); - Mockito.when(FileWriterFactory.create(Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class), Mockito.any(SecorConfig.class))).thenReturn( - writer); + PowerMockito.mockStatic(ReflectionUtil.class); + FileReaderWriter writer = Mockito.mock(FileReaderWriter.class); + Mockito.when( + ReflectionUtil.createFileReaderWriter( + Mockito.any(String.class), + Mockito.any(LogFilePath.class), + Mockito.any(CompressionCodec.class))) + .thenReturn(writer); Mockito.when(writer.getLength()).thenReturn(123L); - FileWriter createdWriter = mRegistry.getOrCreateWriter(mLogFilePath, null); + FileReaderWriter createdWriter = mRegistry.getOrCreateWriter( + mLogFilePath, null); assertTrue(createdWriter == 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(); - FileWriterFactory.create(Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class), Mockito.any(SecorConfig.class)); + ReflectionUtil.createFileReaderWriter(Mockito.any(String.class), + Mockito.any(LogFilePath.class), + Mockito.any(CompressionCodec.class)); PowerMockito.verifyStatic(); FileUtil.delete(PATH); @@ -98,11 +101,13 @@ public void testGetOrCreateWriter() throws Exception { FileUtil.delete(CRC_PATH); 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)); } @@ -110,15 +115,19 @@ public void testGetOrCreateWriter() throws Exception { private void createCompressedWriter() throws Exception { PowerMockito.mockStatic(FileUtil.class); - PowerMockito.mockStatic(FileWriterFactory.class); - FileWriter writer = Mockito.mock(FileWriter.class); - Mockito.when(FileWriterFactory.create(Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class), Mockito.any(SecorConfig.class))).thenReturn( - writer); + PowerMockito.mockStatic(ReflectionUtil.class); + FileReaderWriter writer = Mockito.mock(FileReaderWriter.class); + Mockito.when( + ReflectionUtil.createFileReaderWriter( + Mockito.any(String.class), + Mockito.any(LogFilePath.class), + Mockito.any(CompressionCodec.class))) + .thenReturn(writer); Mockito.when(writer.getLength()).thenReturn(123L); - FileWriter createdWriter = mRegistry.getOrCreateWriter(mLogFilePathGz, new GzipCodec()); + FileReaderWriter createdWriter = mRegistry.getOrCreateWriter( + mLogFilePathGz, new GzipCodec()); assertTrue(createdWriter == writer); } @@ -134,15 +143,18 @@ public void testGetOrCreateWriterCompressed() throws Exception { FileUtil.delete(CRC_PATH); PowerMockito.verifyStatic(); - FileWriterFactory.create(Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class), Mockito.any(SecorConfig.class)); + ReflectionUtil.createFileReaderWriter(Mockito.any(String.class), + Mockito.any(LogFilePath.class), + Mockito.any(CompressionCodec.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)); } @@ -185,7 +197,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/io/FileReaderTest.java b/src/test/java/com/pinterest/secor/io/FileReaderTest.java deleted file mode 100644 index 36c5e08dd..000000000 --- a/src/test/java/com/pinterest/secor/io/FileReaderTest.java +++ /dev/null @@ -1,134 +0,0 @@ -/** - * 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.FileInputStream; -import java.io.InputStream; -import java.util.zip.GZIPInputStream; - -import org.apache.commons.configuration.PropertiesConfiguration; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.SequenceFile; -import org.junit.runner.RunWith; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; - -import com.pinterest.secor.common.LogFilePath; -import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.io.impl.GzipFileReader; -import com.pinterest.secor.io.impl.SequenceFileReader; - -import junit.framework.TestCase; - -/** - * Test the file readers - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -@RunWith(PowerMockRunner.class) -@PrepareForTest({ FileSystem.class, GzipFileReader.class, SequenceFileReader.class }) -public class FileReaderTest 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 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", - "com.pinterest.secor.io.impl.SequenceFileReader"); - mConfig = new SecorConfig(properties); - } - - private void setupGzipFileReaderConfig() { - PropertiesConfiguration properties = new PropertiesConfiguration(); - properties.addProperty("secor.file.reader", - "com.pinterest.secor.io.impl.GzipFileReader"); - mConfig = new SecorConfig(properties); - } - - public void testSequenceFileReader() throws Exception { - setupSequenceFileReaderConfig(); - mockSequenceFileReader(); - - FileReaderFactory.create(mLogFilePath, mConfig); - - // Verify that the method has been called exactly once (the default). - PowerMockito.verifyStatic(); - FileSystem.get(Mockito.any(Configuration.class)); - } - - public void testGzipFileReader() throws Exception { - setupGzipFileReaderConfig(); - mockGzipFileReader(); - FileReaderFactory.create(mLogFilePathGz, mConfig); - } - - private void mockGzipFileReader() throws Exception { - FileInputStream fileInputStreamMock = PowerMockito - .mock(FileInputStream.class); - - GZIPInputStream gzipInputStreamMock = PowerMockito - .mock(GZIPInputStream.class); - - PowerMockito.whenNew(FileInputStream.class) - .withParameterTypes(String.class) - .withArguments(Mockito.any(String.class)) - .thenReturn(fileInputStreamMock); - - PowerMockito.whenNew(GZIPInputStream.class) - .withParameterTypes(InputStream.class) - .withArguments(Mockito.any(FileInputStream.class)) - .thenReturn(gzipInputStreamMock); - } - - private void mockSequenceFileReader() throws Exception { - PowerMockito.mockStatic(FileSystem.class); - FileSystem fs = Mockito.mock(FileSystem.class); - Mockito.when(FileSystem.get(Mockito.any(Configuration.class))) - .thenReturn(fs); - - Path fsPath = new Path(PATH); - 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); - } -} diff --git a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java new file mode 100644 index 000000000..2e4a58b69 --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java @@ -0,0 +1,205 @@ +/** + * 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.InputStream; +import java.io.OutputStream; + +import org.apache.commons.configuration.PropertiesConfiguration; +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.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.DelimitedTextFileReaderWriter; +import com.pinterest.secor.io.impl.SequenceFileReaderWriter; +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, DelimitedTextFileReaderWriter.class, + SequenceFile.class, SequenceFileReaderWriter.class, GzipCodec.class }) +public class FileReaderWriterTest 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 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", + "com.pinterest.secor.io.impl.SequenceFileReaderWriter"); + mConfig = new SecorConfig(properties); + } + + private void setupGzipFileReaderConfig() { + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.addProperty("secor.file.reader.writer", + "com.pinterest.secor.io.impl.DelimitedTextFileReaderWriter"); + mConfig = new SecorConfig(properties); + } + + private void mockGzipFileReaderWriter() throws Exception { + GzipCodec codec = PowerMockito.mock(GzipCodec.class); + PowerMockito.whenNew(GzipCodec.class).withNoArguments() + .thenReturn(codec); + + 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 mockSequenceFileReaderWriter(boolean isCompressed) + throws Exception { + PowerMockito.mockStatic(FileSystem.class); + FileSystem fs = Mockito.mock(FileSystem.class); + Mockito.when(FileSystem.get(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(); + mockSequenceFileReaderWriter(false); + ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), + mLogFilePath, null); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem.get(Mockito.any(Configuration.class)); + } + + public void testSequenceFileWriter() throws Exception { + setupSequenceFileReaderConfig(); + mockSequenceFileReaderWriter(false); + + FileReaderWriter writer = (FileReaderWriter) ReflectionUtil + .createFileReaderWriter(mConfig.getFileReaderWriter(), + mLogFilePath, null); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem.get(Mockito.any(Configuration.class)); + + assert writer.getLength() == 123L; + + mockSequenceFileReaderWriter(true); + + ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), + mLogFilePathGz, new GzipCodec()); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem.get(Mockito.any(Configuration.class)); + + assert writer.getLength() == 12L; + } + + public void testGzipFileWriter() throws Exception { + setupGzipFileReaderConfig(); + mockGzipFileReaderWriter(); + FileReaderWriter writer = (FileReaderWriter) ReflectionUtil + .createFileReaderWriter(mConfig.getFileReaderWriter(), + mLogFilePathGz, new GzipCodec()); + assert writer.getLength() == 0L; + } + + public void testGzipFileReader() throws Exception { + setupGzipFileReaderConfig(); + mockGzipFileReaderWriter(); + ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), + mLogFilePathGz, new GzipCodec()); + } +} diff --git a/src/test/java/com/pinterest/secor/io/FileWriterTest.java b/src/test/java/com/pinterest/secor/io/FileWriterTest.java deleted file mode 100644 index 0de64eac1..000000000 --- a/src/test/java/com/pinterest/secor/io/FileWriterTest.java +++ /dev/null @@ -1,162 +0,0 @@ -/** - * 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.FileOutputStream; -import java.io.IOException; - -import junit.framework.TestCase; - -import org.apache.commons.configuration.PropertiesConfiguration; -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.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.GzipFileWriter; - -/** - * - * Test the file writers - * - * @author Praveen Murugesan (praveen@uber.com) - * - */ -@RunWith(PowerMockRunner.class) -@PrepareForTest({ FileSystem.class, SequenceFile.class, GzipFileWriter.class }) -public class FileWriterTest 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 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 setupSequenceFileWriterConfig() { - PropertiesConfiguration properties = new PropertiesConfiguration(); - properties.addProperty("secor.file.writer", - "com.pinterest.secor.io.impl.SequenceFileWriter"); - mConfig = new SecorConfig(properties); - } - - private void setupGzipFileWriterConfig() { - PropertiesConfiguration properties = new PropertiesConfiguration(); - properties.addProperty("secor.file.writer", - "com.pinterest.secor.io.impl.GzipFileWriter"); - mConfig = new SecorConfig(properties); - } - - public void testSequenceFileWriter() throws Exception { - setupSequenceFileWriterConfig(); - mockSequenceFileWriter(); - - FileWriter writer = FileWriterFactory.create(mLogFilePath, null, - mConfig); - - // Verify that the method has been called exactly once (the default). - PowerMockito.verifyStatic(); - FileSystem.get(Mockito.any(Configuration.class)); - - assert writer.getLength() == 123L; - - mockSequenceCompressedFileWriter(); - - FileWriterFactory.create(mLogFilePathGz, new GzipCodec(), mConfig); - - // Verify that the method has been called exactly once (the default). - PowerMockito.verifyStatic(); - FileSystem.get(Mockito.any(Configuration.class)); - - assert writer.getLength() == 12L; - } - - public void testGzipFileWriter() throws Exception { - setupGzipFileWriterConfig(); - mockGzipFileWriter(); - FileWriter writer = FileWriterFactory.create(mLogFilePathGz, null, - mConfig); - assert writer.getLength() == 0L; - } - - private void mockGzipFileWriter() throws Exception { - FileOutputStream fileOutputStreamMock = PowerMockito - .mock(FileOutputStream.class); - - PowerMockito.whenNew(FileOutputStream.class) - .withParameterTypes(String.class) - .withArguments(Mockito.any(String.class)) - .thenReturn(fileOutputStreamMock); - } - - private void mockSequenceFileWriter() throws IOException { - PowerMockito.mockStatic(FileSystem.class); - FileSystem fs = Mockito.mock(FileSystem.class); - Mockito.when(FileSystem.get(Mockito.any(Configuration.class))) - .thenReturn(fs); - - PowerMockito.mockStatic(SequenceFile.class); - Path fsPath = new Path(PATH); - SequenceFile.Writer writer = Mockito.mock(SequenceFile.Writer.class); - Mockito.when( - SequenceFile.createWriter(Mockito.eq(fs), - Mockito.any(Configuration.class), Mockito.eq(fsPath), - Mockito.eq(LongWritable.class), - Mockito.eq(BytesWritable.class))).thenReturn(writer); - - Mockito.when(writer.getLength()).thenReturn(123L); - } - - private void mockSequenceCompressedFileWriter() throws IOException { - 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); - - Mockito.when(writer.getLength()).thenReturn(12L); - } -} diff --git a/src/test/java/com/pinterest/secor/performance/PerformanceTest.java b/src/test/java/com/pinterest/secor/performance/PerformanceTest.java index c56fece72..59dd1e434 100644 --- a/src/test/java/com/pinterest/secor/performance/PerformanceTest.java +++ b/src/test/java/com/pinterest/secor/performance/PerformanceTest.java @@ -29,6 +29,7 @@ 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; @@ -63,201 +64,209 @@ */ public class PerformanceTest { - public static void main(String[] args) throws Exception { - if (args.length != 4) { - System.err.println("USAGE: java " + PerformanceTest.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)); - } + public static void main(String[] args) throws Exception { + if (args.length != 4) { + System.err.println("USAGE: java " + PerformanceTest.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]); + int num_partitions = Integer.parseInt(args[1]); - // createTopics - createTopics(topics, num_partitions, zkConfig); + // 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"); + 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); + ProducerConfig producerConfig = new ProducerConfig(props); - Producer producer = new Producer( - producerConfig); - long size = 0; - int message_size = Integer.parseInt(args[3]); + 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(); + // 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); + 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(); - } + 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++) { - long val = consumer.getOffset(topic, i); - 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)); - } - } - } - } + 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; - // 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.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)); + } + } + } + } - System.exit(1); - } + // 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; + } + } - /** - * 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); + System.exit(1); + } - 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; - } + /** + * 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); - /** - * Helper to create topics - * @param topics - * @param partitions - * @param zkConfig - * @throws InterruptedException - */ - private static void createTopics(List topics, int partitions, - String zkConfig) throws InterruptedException { + 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; + } - ZkClient zkClient = createZkClient(zkConfig); + /** + * Helper to create topics + * + * @param topics + * @param partitions + * @param zkConfig + * @throws InterruptedException + */ + private static void createTopics(List topics, int partitions, + String zkConfig) throws InterruptedException { - 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(); - } + 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; - } + } + + /** + * 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/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index 269cba6f0..fc3d3b6fc 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -17,13 +17,14 @@ 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.FileReaderWriter; import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.IdUtil; import junit.framework.TestCase; + +import org.apache.hadoop.io.compress.CompressionCodec; import org.junit.runner.RunWith; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -41,23 +42,24 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ @RunWith(PowerMockRunner.class) -@PrepareForTest({FileUtil.class, IdUtil.class}) +@PrepareForTest({ FileUtil.class, IdUtil.class }) public class UploaderTest extends TestCase { private static class TestUploader extends Uploader { - private FileReader mReader; + private FileReaderWriter mReader; public TestUploader(SecorConfig config, OffsetTracker offsetTracker, - FileRegistry fileRegistry, ZookeeperConnector zookeeperConnector) { + FileRegistry fileRegistry, ZookeeperConnector zookeeperConnector) { super(config, offsetTracker, fileRegistry, zookeeperConnector); - mReader = Mockito.mock(FileReader.class); + mReader = Mockito.mock(FileReaderWriter.class); } @Override - protected FileReader createReader(LogFilePath srcPath) throws IOException { + protected FileReaderWriter createReader(LogFilePath srcPath, + CompressionCodec codec) throws IOException { return mReader; } - public FileReader getReader() { + public FileReaderWriter getReader() { return mReader; } } @@ -79,8 +81,8 @@ 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"); @@ -92,24 +94,36 @@ 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); mZookeeperConnector = Mockito.mock(ZookeeperConnector.class); - mUploader = new TestUploader(mConfig, mOffsetTracker, mFileRegistry, mZookeeperConnector); + mUploader = new TestUploader(mConfig, mOffsetTracker, mFileRegistry, + mZookeeperConnector); } public void testUploadFiles() throws Exception { - Mockito.when(mZookeeperConnector.getCommittedOffsetCount(mTopicPartition)).thenReturn(11L); - Mockito.when(mOffsetTracker.setCommittedOffsetCount(mTopicPartition, 11L)).thenReturn(11L); - Mockito.when(mOffsetTracker.setCommittedOffsetCount(mTopicPartition, 21L)).thenReturn(11L); - Mockito.when(mOffsetTracker.getLastSeenOffset(mTopicPartition)).thenReturn(20L); - Mockito.when(mOffsetTracker.getTrueCommittedOffsetCount(mTopicPartition)).thenReturn(11L); + Mockito.when( + mZookeeperConnector.getCommittedOffsetCount(mTopicPartition)) + .thenReturn(11L); + Mockito.when( + mOffsetTracker.setCommittedOffsetCount(mTopicPartition, 11L)) + .thenReturn(11L); + Mockito.when( + mOffsetTracker.setCommittedOffsetCount(mTopicPartition, 21L)) + .thenReturn(11L); + Mockito.when(mOffsetTracker.getLastSeenOffset(mTopicPartition)) + .thenReturn(20L); + Mockito.when( + mOffsetTracker.getTrueCommittedOffsetCount(mTopicPartition)) + .thenReturn(11L); Mockito.when(mConfig.getS3Bucket()).thenReturn("some_bucket"); Mockito.when(mConfig.getS3Path()).thenReturn("some_s3_parent_dir"); HashSet logFilePaths = new HashSet(); logFilePaths.add(mLogFilePath); - Mockito.when(mFileRegistry.getPaths(mTopicPartition)).thenReturn(logFilePaths); + Mockito.when(mFileRegistry.getPaths(mTopicPartition)).thenReturn( + logFilePaths); PowerMockito.mockStatic(FileUtil.class); @@ -119,20 +133,27 @@ public void testUploadFiles() throws Exception { 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"); + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000010", + "s3n://some_bucket/some_s3_parent_dir/some_topic/some_partition/" + + "some_other_partition/10_0_00000000000000000010"); Mockito.verify(mFileRegistry).deleteTopicPartition(mTopicPartition); - Mockito.verify(mZookeeperConnector).setCommittedOffsetCount(mTopicPartition, 21L); - Mockito.verify(mOffsetTracker).setCommittedOffsetCount(mTopicPartition, 21L); + Mockito.verify(mZookeeperConnector).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(); @@ -140,20 +161,28 @@ 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); - FileReader reader = mUploader.getReader(); + FileReaderWriter reader = mUploader.getReader(); Mockito.when(reader.next()).thenAnswer(new Answer() { private int mCallCount = 0; + @Override - public KeyValue answer(InvocationOnMock invocation) throws Throwable { + public KeyValue answer(InvocationOnMock invocation) + throws Throwable { if (mCallCount == 2) { return null; } @@ -162,18 +191,21 @@ public KeyValue answer(InvocationOnMock invocation) throws Throwable { }); PowerMockito.mockStatic(IdUtil.class); - Mockito.when(IdUtil.getLocalMessageDir()).thenReturn("some_message_dir"); + Mockito.when(IdUtil.getLocalMessageDir()) + .thenReturn("some_message_dir"); - 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); + FileReaderWriter writer = Mockito.mock(FileReaderWriter.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).write(Mockito.any(long.class), - Mockito.any(byte[].class)); + Mockito.any(byte[].class)); Mockito.verify(mFileRegistry).deletePath(mLogFilePath); } } From 094a37628cf8b087f914d5a0b7b193f159110df2 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Mon, 6 Oct 2014 21:46:30 -0700 Subject: [PATCH 013/330] add param to specify reader/writer --- .../pinterest/secor/common/FileRegistry.java | 4 +- .../pinterest/secor/io/FileReaderWriter.java | 5 +++ .../impl/DelimitedTextFileReaderWriter.java | 37 +++++++++++++------ .../io/impl/SequenceFileReaderWriter.java | 33 +++++++++++------ .../secor/parser/PartitionFinalizer.java | 4 +- .../pinterest/secor/uploader/Uploader.java | 6 ++- .../pinterest/secor/util/ReflectionUtil.java | 13 +++---- .../pinterest/secor/writer/MessageWriter.java | 3 +- .../secor/common/FileRegistryTest.java | 12 ++++-- .../secor/io/FileReaderWriterTest.java | 10 ++--- 10 files changed, 81 insertions(+), 46 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index 64bb9e45c..ed584593f 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -99,7 +99,9 @@ public FileReaderWriter getOrCreateWriter(LogFilePath path, CompressionCodec cod if (!files.contains(path)) { files.add(path); } - writer = ((FileReaderWriter) ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), path, codec)); + writer = ((FileReaderWriter) ReflectionUtil.createFileReaderWriter( + mConfig.getFileReaderWriter(), path, codec, + FileReaderWriter.Type.Writer)); mWriters.put(path, writer); mCreationTimes.put(path, System.currentTimeMillis() / 1000L); LOG.debug("created writer for path " + path.getLogFilePath()); diff --git a/src/main/java/com/pinterest/secor/io/FileReaderWriter.java b/src/main/java/com/pinterest/secor/io/FileReaderWriter.java index 15b105ed8..0f72e25bd 100644 --- a/src/main/java/com/pinterest/secor/io/FileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/FileReaderWriter.java @@ -25,6 +25,11 @@ * */ public interface FileReaderWriter { + + public enum Type { + Reader, + Writer; + } /** * Get the next key/value from the file diff --git a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java index 9c11cf5fc..91eadbeb0 100644 --- a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java @@ -53,20 +53,30 @@ public class DelimitedTextFileReaderWriter implements FileReaderWriter { // constructor public DelimitedTextFileReaderWriter(LogFilePath path, - CompressionCodec codec) throws FileNotFoundException, IOException { + CompressionCodec codec, FileReaderWriter.Type type) + throws FileNotFoundException, IOException { + File logFile = new File(path.getLogFilePath()); logFile.getParentFile().mkdirs(); - this.countingStream = new CountingOutputStream(new FileOutputStream( - logFile)); - this.writer = (codec == null) ? new BufferedOutputStream( - this.countingStream) : new BufferedOutputStream( - codec.createOutputStream(this.countingStream)); - - InputStream inputStream = new FileInputStream(new File( - path.getLogFilePath())); - this.reader = (codec == null) ? new BufferedInputStream(inputStream) - : new BufferedInputStream(codec.createInputStream(inputStream)); - this.offset = path.getOffset(); + if (type == FileReaderWriter.Type.Reader) { + InputStream inputStream = new FileInputStream(new File( + path.getLogFilePath())); + this.reader = (codec == null) ? new BufferedInputStream(inputStream) + : new BufferedInputStream( + codec.createInputStream(inputStream)); + this.offset = path.getOffset(); + this.countingStream = null; + this.writer = null; + } else if (type == FileReaderWriter.Type.Writer) { + this.countingStream = new CountingOutputStream( + new FileOutputStream(logFile)); + this.writer = (codec == null) ? new BufferedOutputStream( + this.countingStream) : new BufferedOutputStream( + codec.createOutputStream(this.countingStream)); + this.reader = null; + } else { + throw new IllegalArgumentException("Undefined File Type: " + type); + } } @Override @@ -81,17 +91,20 @@ public void close() throws IOException { @Override public long getLength() throws IOException { + assert this.countingStream != null; return this.countingStream.getCount(); } @Override public void write(long key, byte[] value) throws IOException { + assert this.writer != null; this.writer.write(value); this.writer.write(DELIMITER); } @Override public KeyValue next() throws IOException { + assert this.reader != null; ByteArrayOutputStream messageBuffer = new ByteArrayOutputStream(); int nextByte; while ((nextByte = reader.read()) != DELIMITER) { diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java index 974232220..3105d6230 100644 --- a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java @@ -45,22 +45,33 @@ public class SequenceFileReaderWriter implements FileReaderWriter { private final BytesWritable value; // constructor - public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec) - throws Exception { + public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec, + FileReaderWriter.Type type) throws Exception { Configuration config = new Configuration(); FileSystem fs = FileSystem.get(config); Path fsPath = new Path(path.getLogFilePath()); - if (codec != null) { - this.writer = SequenceFile.createWriter(fs, config, fsPath, - LongWritable.class, BytesWritable.class, - SequenceFile.CompressionType.BLOCK, codec); + + if (type == FileReaderWriter.Type.Reader) { + this.reader = new SequenceFile.Reader(fs, fsPath, config); + this.key = (LongWritable) reader.getKeyClass().newInstance(); + this.value = (BytesWritable) reader.getValueClass().newInstance(); + this.writer = null; + } else if (type == FileReaderWriter.Type.Writer) { + if (codec != null) { + this.writer = SequenceFile.createWriter(fs, config, fsPath, + LongWritable.class, BytesWritable.class, + SequenceFile.CompressionType.BLOCK, codec); + } else { + this.writer = SequenceFile.createWriter(fs, config, fsPath, + LongWritable.class, BytesWritable.class); + } + this.reader = null; + this.key = null; + this.value = null; } else { - this.writer = SequenceFile.createWriter(fs, config, fsPath, - LongWritable.class, BytesWritable.class); + throw new IllegalArgumentException("Undefined File Type: " + type); } - this.reader = new SequenceFile.Reader(fs, fsPath, config); - this.key = (LongWritable) reader.getKeyClass().newInstance(); - this.value = (BytesWritable) reader.getValueClass().newInstance(); + } @Override diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 9f03e18da..458b9855d 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -18,8 +18,8 @@ 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; @@ -56,7 +56,7 @@ public PartitionFinalizer(SecorConfig config) throws Exception { mThriftMessageParser = new ThriftMessageParser(mConfig); mQuboleClient = new QuboleClient(mConfig); if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { - CompressionCodec codec = ((CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec())); + CompressionCodec codec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); mFileExtension = codec.getDefaultExtension(); } else { mFileExtension = ""; diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index a5c3e4822..d579c8318 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -19,6 +19,7 @@ import com.pinterest.secor.common.*; import com.pinterest.secor.io.FileReaderWriter; import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.CompressionUtil; import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.IdUtil; import com.pinterest.secor.util.ReflectionUtil; @@ -105,7 +106,8 @@ protected FileReaderWriter createReader(LogFilePath srcPath, CompressionCodec co return (FileReaderWriter) ReflectionUtil.createFileReaderWriter( mConfig.getFileReaderWriter(), srcPath, - codec); + codec, + FileReaderWriter.Type.Reader); } private void trim(LogFilePath srcPath, long startOffset) throws Exception { @@ -122,7 +124,7 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { 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(); } reader = createReader(srcPath, codec); diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 09fb7fc3e..5f93a94a6 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -18,6 +18,7 @@ import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.FileReaderWriter; import java.lang.reflect.Constructor; @@ -30,11 +31,6 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class ReflectionUtil { - public static Object createCompressionCodec(String className) - throws Exception { - Class clazz = Class.forName(className); - return clazz.getConstructor().newInstance(); - } public static Object createMessageParser(String className, SecorConfig config) throws Exception { @@ -54,7 +50,8 @@ public static Object createMessageParser(String className, } public static Object createFileReaderWriter(String className, - LogFilePath logFilePath, CompressionCodec compressionCodec) + LogFilePath logFilePath, CompressionCodec compressionCodec, + FileReaderWriter.Type type) throws Exception { Class clazz = Class.forName(className); // Search for an "appropriate" constructor. @@ -62,8 +59,8 @@ public static Object createFileReaderWriter(String className, Class[] paramTypes = ctor.getParameterTypes(); // If the arity matches, let's use it. - if (paramTypes.length == 2) { - Object[] args = { logFilePath, compressionCodec }; + if (paramTypes.length == 3) { + Object[] args = { logFilePath, compressionCodec, type }; return ctor.newInstance(args); } } diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index 2af1b1ff0..72ff01ea7 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -24,6 +24,7 @@ import java.io.IOException; +import com.pinterest.secor.util.CompressionUtil; import com.pinterest.secor.util.IdUtil; import com.pinterest.secor.util.ReflectionUtil; @@ -52,7 +53,7 @@ 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 { mFileExtension = ""; diff --git a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java index 08eef78a7..8aed5d0ca 100644 --- a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java +++ b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java @@ -73,7 +73,8 @@ private void createWriter() throws Exception { ReflectionUtil.createFileReaderWriter( Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class))) + Mockito.any(CompressionCodec.class), + Mockito.any(FileReaderWriter.Type.class))) .thenReturn(writer); Mockito.when(writer.getLength()).thenReturn(123L); @@ -93,7 +94,8 @@ public void testGetOrCreateWriter() throws Exception { PowerMockito.verifyStatic(); ReflectionUtil.createFileReaderWriter(Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class)); + Mockito.any(CompressionCodec.class), + Mockito.any(FileReaderWriter.Type.class)); PowerMockito.verifyStatic(); FileUtil.delete(PATH); @@ -121,7 +123,8 @@ private void createCompressedWriter() throws Exception { ReflectionUtil.createFileReaderWriter( Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class))) + Mockito.any(CompressionCodec.class), + Mockito.any(FileReaderWriter.Type.class))) .thenReturn(writer); Mockito.when(writer.getLength()).thenReturn(123L); @@ -145,7 +148,8 @@ public void testGetOrCreateWriterCompressed() throws Exception { PowerMockito.verifyStatic(); ReflectionUtil.createFileReaderWriter(Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class)); + Mockito.any(CompressionCodec.class), + Mockito.any(FileReaderWriter.Type.class)); TopicPartition topicPartition = new TopicPartition("some_topic", 0); Collection topicPartitions = mRegistry diff --git a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java index 2e4a58b69..0681cfac0 100644 --- a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java +++ b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java @@ -154,7 +154,7 @@ public void testSequenceFileReader() throws Exception { setupSequenceFileReaderConfig(); mockSequenceFileReaderWriter(false); ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePath, null); + mLogFilePath, null, FileReaderWriter.Type.Reader); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); @@ -167,7 +167,7 @@ public void testSequenceFileWriter() throws Exception { FileReaderWriter writer = (FileReaderWriter) ReflectionUtil .createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePath, null); + mLogFilePath, null, FileReaderWriter.Type.Writer); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); @@ -178,7 +178,7 @@ public void testSequenceFileWriter() throws Exception { mockSequenceFileReaderWriter(true); ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePathGz, new GzipCodec()); + mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Writer); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); @@ -192,7 +192,7 @@ public void testGzipFileWriter() throws Exception { mockGzipFileReaderWriter(); FileReaderWriter writer = (FileReaderWriter) ReflectionUtil .createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePathGz, new GzipCodec()); + mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Writer); assert writer.getLength() == 0L; } @@ -200,6 +200,6 @@ public void testGzipFileReader() throws Exception { setupGzipFileReaderConfig(); mockGzipFileReaderWriter(); ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePathGz, new GzipCodec()); + mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Reader); } } From 833ba805a03f23ab3ae789a20d17a1bd1fbb2961 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Mon, 6 Oct 2014 21:46:56 -0700 Subject: [PATCH 014/330] add Compression util method --- .../pinterest/secor/util/CompressionUtil.java | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 src/main/java/com/pinterest/secor/util/CompressionUtil.java 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..c7d4bf708 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/CompressionUtil.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.util; + +import 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 { + CompressionCodecFactory ccf = new CompressionCodecFactory( + new Configuration()); + return ccf.getCodecByClassName(className); + } +} From 098ca46537440c07a2ab414bacd59357722e89f1 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Mon, 6 Oct 2014 21:54:19 -0700 Subject: [PATCH 015/330] NPE fix --- .../com/pinterest/secor/io/impl/SequenceFileReaderWriter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java index 3105d6230..ff721185e 100644 --- a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java @@ -76,10 +76,10 @@ public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec, @Override public void close() throws IOException { - if (this.writer == null) { + if (this.writer != null) { this.writer.close(); } - if (this.reader == null) { + if (this.reader != null) { this.reader.close(); } } From 1b6c1454211faf4946ba77db0ad9631b02c141da Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Tue, 7 Oct 2014 19:12:00 -0700 Subject: [PATCH 016/330] remove unused imports --- src/main/java/com/pinterest/secor/writer/MessageWriter.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index 72ff01ea7..92198da19 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -26,7 +26,6 @@ import com.pinterest.secor.util.CompressionUtil; import com.pinterest.secor.util.IdUtil; -import com.pinterest.secor.util.ReflectionUtil; import org.apache.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; From 05b77db0cbc3c2cb02ef0d240aec2e1bcd52f24b Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Wed, 8 Oct 2014 22:19:56 -0700 Subject: [PATCH 017/330] mock gzip file input/output stream for tests --- .../pinterest/secor/io/FileReaderWriterTest.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java index 0681cfac0..ad01552f7 100644 --- a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java +++ b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java @@ -16,6 +16,8 @@ */ package com.pinterest.secor.io; +import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.InputStream; import java.io.OutputStream; @@ -51,7 +53,8 @@ */ @RunWith(PowerMockRunner.class) @PrepareForTest({ FileSystem.class, DelimitedTextFileReaderWriter.class, - SequenceFile.class, SequenceFileReaderWriter.class, GzipCodec.class }) + SequenceFile.class, SequenceFileReaderWriter.class, GzipCodec.class, + FileInputStream.class, FileOutputStream.class}) public class FileReaderWriterTest extends TestCase { private static final String PATH = "/some_parent_dir/some_topic/some_partition/some_other_partition/" @@ -89,6 +92,15 @@ private void mockGzipFileReaderWriter() throws Exception { PowerMockito.whenNew(GzipCodec.class).withNoArguments() .thenReturn(codec); + FileInputStream fileInputStream = Mockito.mock(FileInputStream.class); + FileOutputStream fileOutputStream = Mockito.mock(FileOutputStream.class); + + PowerMockito.whenNew(FileInputStream.class).withAnyArguments() + .thenReturn(fileInputStream); + + PowerMockito.whenNew(FileOutputStream.class).withAnyArguments() + .thenReturn(fileOutputStream); + CompressionInputStream inputStream = Mockito .mock(CompressionInputStream.class); CompressionOutputStream outputStream = Mockito From cd3b308778b4255ef9b8822941acbed0f2765a68 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Thu, 9 Oct 2014 14:17:29 -0700 Subject: [PATCH 018/330] changes Pawel suggested --- README.md | 1 + .../pinterest/secor/common/SecorConfig.java | 4 -- .../pinterest/secor/io/FileReaderWriter.java | 2 +- .../java/com/pinterest/secor/io/KeyValue.java | 12 ++--- .../impl/DelimitedTextFileReaderWriter.java | 52 +++++++++---------- .../io/impl/SequenceFileReaderWriter.java | 48 ++++++++--------- .../pinterest/secor/uploader/Uploader.java | 2 +- .../pinterest/secor/util/ReflectionUtil.java | 31 ++++++----- .../pinterest/secor/writer/MessageWriter.java | 3 +- .../secor/uploader/UploaderTest.java | 3 +- 10 files changed, 77 insertions(+), 81 deletions(-) diff --git a/README.md b/README.md index 4e2dbb676..4ef260ce5 100644 --- a/README.md +++ b/README.md @@ -102,6 +102,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Brenden Matthews](https://github.com/brndnmtthws) * [Lucas Zago](https://github.com/zago) * [James Green] (https://github.com/jfgreen) + * [Praveen Murugesan] (https://github.com/lefthandmagic) ## Help diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 01926fe9b..479c4bf44 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -198,10 +198,6 @@ public String getMessageTimestampInputPattern() { public String getCompressionCodec() { return getString("secor.compression.codec"); } - - public String getFileExtension() { - return getString("secor.file.extension"); - } public int getMaxMessageSizeBytes() { return getInt("secor.max.message.size.bytes"); diff --git a/src/main/java/com/pinterest/secor/io/FileReaderWriter.java b/src/main/java/com/pinterest/secor/io/FileReaderWriter.java index 0f72e25bd..7b961bde2 100644 --- a/src/main/java/com/pinterest/secor/io/FileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/FileReaderWriter.java @@ -61,6 +61,6 @@ public enum Type { * @param value * @throws IOException */ - public void write(long key, byte[] value) throws IOException; + public void write(KeyValue keyValue) throws IOException; } diff --git a/src/main/java/com/pinterest/secor/io/KeyValue.java b/src/main/java/com/pinterest/secor/io/KeyValue.java index b4ba8fa63..a8e6f11af 100644 --- a/src/main/java/com/pinterest/secor/io/KeyValue.java +++ b/src/main/java/com/pinterest/secor/io/KeyValue.java @@ -25,21 +25,21 @@ */ public class KeyValue { - private final long key; - private final byte[] value; + private final long mKey; + private final byte[] mValue; //constructor public KeyValue(long key, byte[] value) { - this.key = key; - this.value = value; + this.mKey = key; + this.mValue = value; } public long getKey() { - return this.key; + return this.mKey; } public byte[] getValue() { - return this.value; + return this.mValue; } } diff --git a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java index 91eadbeb0..9dcfdaa57 100644 --- a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java @@ -45,11 +45,11 @@ public class DelimitedTextFileReaderWriter implements FileReaderWriter { // delimiter used between messages private static final byte DELIMITER = '\n'; - private final CountingOutputStream countingStream; - private final BufferedOutputStream writer; + private final CountingOutputStream mCountingStream; + private final BufferedOutputStream mWriter; - private final BufferedInputStream reader; - private long offset; + private final BufferedInputStream mReader; + private long mOffset; // constructor public DelimitedTextFileReaderWriter(LogFilePath path, @@ -61,19 +61,19 @@ public DelimitedTextFileReaderWriter(LogFilePath path, if (type == FileReaderWriter.Type.Reader) { InputStream inputStream = new FileInputStream(new File( path.getLogFilePath())); - this.reader = (codec == null) ? new BufferedInputStream(inputStream) + this.mReader = (codec == null) ? new BufferedInputStream(inputStream) : new BufferedInputStream( codec.createInputStream(inputStream)); - this.offset = path.getOffset(); - this.countingStream = null; - this.writer = null; + this.mOffset = path.getOffset(); + this.mCountingStream = null; + this.mWriter = null; } else if (type == FileReaderWriter.Type.Writer) { - this.countingStream = new CountingOutputStream( + this.mCountingStream = new CountingOutputStream( new FileOutputStream(logFile)); - this.writer = (codec == null) ? new BufferedOutputStream( - this.countingStream) : new BufferedOutputStream( - codec.createOutputStream(this.countingStream)); - this.reader = null; + this.mWriter = (codec == null) ? new BufferedOutputStream( + this.mCountingStream) : new BufferedOutputStream( + codec.createOutputStream(this.mCountingStream)); + this.mReader = null; } else { throw new IllegalArgumentException("Undefined File Type: " + type); } @@ -81,33 +81,33 @@ public DelimitedTextFileReaderWriter(LogFilePath path, @Override public void close() throws IOException { - if (this.writer != null) { - this.writer.close(); + if (this.mWriter != null) { + this.mWriter.close(); } - if (this.reader != null) { - this.reader.close(); + if (this.mReader != null) { + this.mReader.close(); } } @Override public long getLength() throws IOException { - assert this.countingStream != null; - return this.countingStream.getCount(); + assert this.mCountingStream != null; + return this.mCountingStream.getCount(); } @Override - public void write(long key, byte[] value) throws IOException { - assert this.writer != null; - this.writer.write(value); - this.writer.write(DELIMITER); + public void write(KeyValue keyValue) throws IOException { + assert this.mWriter != null; + this.mWriter.write(keyValue.getValue()); + this.mWriter.write(DELIMITER); } @Override public KeyValue next() throws IOException { - assert this.reader != null; + assert this.mReader != null; ByteArrayOutputStream messageBuffer = new ByteArrayOutputStream(); int nextByte; - while ((nextByte = reader.read()) != DELIMITER) { + while ((nextByte = mReader.read()) != DELIMITER) { if (nextByte == -1) { // end of stream? if (messageBuffer.size() == 0) { // if no byte read return null; @@ -118,7 +118,7 @@ public KeyValue next() throws IOException { } messageBuffer.write(nextByte); } - return new KeyValue(this.offset++, messageBuffer.toByteArray()); + return new KeyValue(this.mOffset++, messageBuffer.toByteArray()); } } diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java index ff721185e..fa5d67d8f 100644 --- a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java @@ -39,10 +39,10 @@ */ public class SequenceFileReaderWriter implements FileReaderWriter { - private final SequenceFile.Writer writer; - private final SequenceFile.Reader reader; - private final LongWritable key; - private final BytesWritable value; + private final SequenceFile.Writer mWriter; + private final SequenceFile.Reader mReader; + private final LongWritable mKey; + private final BytesWritable mValue; // constructor public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec, @@ -52,22 +52,22 @@ public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec, Path fsPath = new Path(path.getLogFilePath()); if (type == FileReaderWriter.Type.Reader) { - this.reader = new SequenceFile.Reader(fs, fsPath, config); - this.key = (LongWritable) reader.getKeyClass().newInstance(); - this.value = (BytesWritable) reader.getValueClass().newInstance(); - this.writer = null; + this.mReader = new SequenceFile.Reader(fs, fsPath, config); + this.mKey = (LongWritable) mReader.getKeyClass().newInstance(); + this.mValue = (BytesWritable) mReader.getValueClass().newInstance(); + this.mWriter = null; } else if (type == FileReaderWriter.Type.Writer) { if (codec != null) { - this.writer = SequenceFile.createWriter(fs, config, fsPath, + this.mWriter = SequenceFile.createWriter(fs, config, fsPath, LongWritable.class, BytesWritable.class, SequenceFile.CompressionType.BLOCK, codec); } else { - this.writer = SequenceFile.createWriter(fs, config, fsPath, + this.mWriter = SequenceFile.createWriter(fs, config, fsPath, LongWritable.class, BytesWritable.class); } - this.reader = null; - this.key = null; - this.value = null; + this.mReader = null; + this.mKey = null; + this.mValue = null; } else { throw new IllegalArgumentException("Undefined File Type: " + type); } @@ -76,30 +76,30 @@ public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec, @Override public void close() throws IOException { - if (this.writer != null) { - this.writer.close(); + if (this.mWriter != null) { + this.mWriter.close(); } - if (this.reader != null) { - this.reader.close(); + if (this.mReader != null) { + this.mReader.close(); } } @Override public long getLength() throws IOException { - return this.writer.getLength(); + return this.mWriter.getLength(); } @Override - public void write(long key, byte[] value) throws IOException { - LongWritable writeableKey = new LongWritable(key); - BytesWritable writeableValue = new BytesWritable(value); - this.writer.append(writeableKey, writeableValue); + public void write(KeyValue keyValue) throws IOException { + LongWritable writeableKey = new LongWritable(keyValue.getKey()); + BytesWritable writeableValue = new BytesWritable(keyValue.getValue()); + this.mWriter.append(writeableKey, writeableValue); } @Override public KeyValue next() throws IOException { - if (reader.next(key, value)) { - return new KeyValue(key.get(), value.getBytes()); + if (mReader.next(mKey, mValue)) { + return new KeyValue(mKey.get(), mValue.getBytes()); } else { return null; } diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index d579c8318..bca7a19c7 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -141,7 +141,7 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { writer = mFileRegistry.getOrCreateWriter(dstPath, codec); } - writer.write(keyVal.getKey(), keyVal.getValue()); + writer.write(keyVal); copiedMessages++; } } diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 5f93a94a6..10aa678ea 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -32,27 +32,26 @@ */ public class ReflectionUtil { - public static Object createMessageParser(String className, - SecorConfig config) throws Exception { - Class clazz = Class.forName(className); + public static Object createMessageParser(String className, + SecorConfig config) throws Exception { + Class clazz = Class.forName(className); - // Search for an "appropriate" constructor. - for (Constructor ctor : clazz.getConstructors()) { - Class[] paramTypes = ctor.getParameterTypes(); + // Search for an "appropriate" constructor. + for (Constructor ctor : clazz.getConstructors()) { + Class[] paramTypes = ctor.getParameterTypes(); - // If the arity matches, let's use it. - if (paramTypes.length == 1) { - Object[] args = { config }; - return ctor.newInstance(args); - } - } - throw new IllegalArgumentException("Class not found " + className); - } + // If the arity matches, let's use it. + if (paramTypes.length == 1) { + Object[] args = { config }; + return ctor.newInstance(args); + } + } + throw new IllegalArgumentException("Class not found " + className); + } public static Object createFileReaderWriter(String className, LogFilePath logFilePath, CompressionCodec compressionCodec, - FileReaderWriter.Type type) - throws Exception { + FileReaderWriter.Type type) throws Exception { Class clazz = Class.forName(className); // Search for an "appropriate" constructor. for (Constructor ctor : clazz.getConstructors()) { diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index 92198da19..80c81cde6 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -18,6 +18,7 @@ import com.pinterest.secor.common.*; import com.pinterest.secor.io.FileReaderWriter; +import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; @@ -82,7 +83,7 @@ public void write(ParsedMessage message) throws Exception { LogFilePath path = new LogFilePath(mLocalPrefix, mConfig.getGeneration(), offset, message, mFileExtension); FileReaderWriter writer = mFileRegistry.getOrCreateWriter(path, mCodec); - writer.write(message.getOffset(), message.getPayload()); + writer.write(new KeyValue(message.getOffset(), message.getPayload())); LOG.debug("appended message " + message + " to file " + path.getLogFilePath() + ". File length " + writer.getLength()); } diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index fc3d3b6fc..3eb2ef9e4 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -204,8 +204,7 @@ public KeyValue answer(InvocationOnMock invocation) mUploader.applyPolicy(); - Mockito.verify(writer).write(Mockito.any(long.class), - Mockito.any(byte[].class)); + Mockito.verify(writer).write(Mockito.any(KeyValue.class)); Mockito.verify(mFileRegistry).deletePath(mLogFilePath); } } From 1c9c7086b7481b7a9d1ed561a8574833d20eabd0 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Fri, 10 Oct 2014 08:05:44 -0700 Subject: [PATCH 019/330] enhance comment for file reader writer interface --- src/main/java/com/pinterest/secor/io/FileReaderWriter.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/main/java/com/pinterest/secor/io/FileReaderWriter.java b/src/main/java/com/pinterest/secor/io/FileReaderWriter.java index 7b961bde2..292af274d 100644 --- a/src/main/java/com/pinterest/secor/io/FileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/FileReaderWriter.java @@ -18,9 +18,16 @@ import java.io.IOException; + /** * Generic file reader/writer interface for all secor files * + * All implementations should define the constructor with the signature: + * FileReaderWriter(LogFilePath path, CompressionCodec codec, FileReaderWriter.Type type) + * + * example: public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec, + * FileReaderWriter.Type type) + * * @author Praveen Murugesan (praveen@uber.com) * */ From 51bce65f5fd56e33691c45023f5f4ab1e99c9cef Mon Sep 17 00:00:00 2001 From: Neil Fulwiler Date: Wed, 22 Oct 2014 21:43:40 -0700 Subject: [PATCH 020/330] mulithread the uploading --- .../pinterest/secor/uploader/Uploader.java | 36 ++++++++++++++----- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index bca7a19c7..76ea305ff 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -28,7 +28,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; /** * Uploader applies a set of policies to determine if any of the locally stored files should be @@ -39,6 +43,8 @@ public class Uploader { private static final Logger LOG = LoggerFactory.getLogger(Uploader.class); + private static final ExecutorService executor = Executors.newFixedThreadPool(256); + private SecorConfig mConfig; private OffsetTracker mOffsetTracker; private FileRegistry mFileRegistry; @@ -57,7 +63,7 @@ public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fi mZookeeperConnector = zookeeperConnector; } - private void upload(LogFilePath localPath) throws Exception { + private Future upload(LogFilePath localPath) throws Exception { String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); LogFilePath s3Path = new LogFilePath(s3Prefix, localPath.getTopic(), localPath.getPartitions(), @@ -65,9 +71,19 @@ private void upload(LogFilePath localPath) throws Exception { localPath.getKafkaPartition(), localPath.getOffset(), localPath.getExtension()); - String localLogFilename = localPath.getLogFilePath(); - LOG.info("uploading file " + localLogFilename + " to " + s3Path.getLogFilePath()); - FileUtil.moveToS3(localLogFilename, s3Path.getLogFilePath()); + final String localLogFilename = localPath.getLogFilePath(); + final String s3LogFilename = s3Path.getLogFilePath(); + LOG.info("uploading file " + localLogFilename + " to " + s3LogFilename); + return executor.submit(new Runnable() { + @Override + public void run() { + try { + FileUtil.moveToS3(localLogFilename, s3LogFilename); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); } private void uploadFiles(TopicPartition topicPartition) throws Exception { @@ -86,8 +102,12 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { LOG.info("uploading topic " + topicPartition.getTopic() + " partition " + topicPartition.getPartition()); Collection paths = mFileRegistry.getPaths(topicPartition); + List> uploadFutures = new ArrayList>(); for (LogFilePath path : paths) { - upload(path); + uploadFutures.add(upload(path)); + } + for (Future uploadFuture : uploadFutures) { + uploadFuture.get(); } mFileRegistry.deleteTopicPartition(topicPartition); mZookeeperConnector.setCommittedOffsetCount(topicPartition, lastSeenOffset + 1); @@ -100,7 +120,7 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { /** * This method is intended to be overwritten in tests. - * @throws Exception + * @throws Exception */ protected FileReaderWriter createReader(LogFilePath srcPath, CompressionCodec codec) throws Exception { return (FileReaderWriter) ReflectionUtil.createFileReaderWriter( @@ -113,7 +133,7 @@ protected FileReaderWriter createReader(LogFilePath srcPath, CompressionCodec co private void trim(LogFilePath srcPath, long startOffset) throws Exception { if (startOffset == srcPath.getOffset()) { return; - } + } FileReaderWriter reader = null; FileReaderWriter writer = null; LogFilePath dstPath = null; @@ -126,7 +146,7 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { codec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); extension = codec.getDefaultExtension(); - } + } reader = createReader(srcPath, codec); KeyValue keyVal; while ((keyVal = reader.next()) != null) { From be7b372c58ead3b8fa2a42c8597adbf925ca1edc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mathias=20So=CC=88derberg?= Date: Sat, 1 Nov 2014 16:40:10 +0100 Subject: [PATCH 021/330] Different log message for new partition MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Currently when Secor starts consuming a “new” partition it logs a warning that the offset for a given partition changed from -2 to *something*, and I’d say that this particular scenario is expected and shouldn’t generate a warning, but rather an info message that consumption of a new partition has started. For future reference, -2 means to start from the earliest available offset and -1 means to start from the latest available offset, if I remember correctly. --- .../com/pinterest/secor/common/OffsetTracker.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/OffsetTracker.java b/src/main/java/com/pinterest/secor/common/OffsetTracker.java index f67a4cc4d..7aa976cf8 100644 --- a/src/main/java/com/pinterest/secor/common/OffsetTracker.java +++ b/src/main/java/com/pinterest/secor/common/OffsetTracker.java @@ -51,9 +51,14 @@ 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 " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition() + " changed from " + lastSeenOffset + " to " + + offset); + } else { + LOG.info("starting to consume topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition() + " from offset " + offset); + } } if (mFirstSeendOffset.get(topicPartition) == null) { mFirstSeendOffset.put(topicPartition, offset); From afd58955dd9000107d616fa66a212e90909b0764 Mon Sep 17 00:00:00 2001 From: Neil Fulwiler Date: Thu, 23 Oct 2014 00:41:43 -0700 Subject: [PATCH 022/330] check upload policy less frequently --- .../pinterest/secor/consumer/Consumer.java | 104 +++++++++++------- 1 file changed, 63 insertions(+), 41 deletions(-) diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index f5cc04d24..73f8ed409 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -82,56 +82,78 @@ 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 nMessages = 0; + long lastChecked = System.currentTimeMillis(); while (true) { - Message rawMessage = null; + boolean hasMoreMessages = consumeNextMessage(); + if (!hasMoreMessages) { + break; + } + + long now = System.currentTimeMillis(); + if (nMessages++ % 10000 == 0 || + (now - lastChecked) > checkEveryNSeconds * 1000) { + lastChecked = now; + checkUploadPolicy(); + } + } + checkUploadPolicy(); + } + + private 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 + private 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; - } - 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); + mMessageWriter.adjustOffset(rawMessage); + } catch (IOException e) { + throw new RuntimeException("Failed to adjust offset.", e); } - if (rawMessage != null) { - // Before parsing, update the offset and remove any redundant data - try { - mMessageWriter.adjustOffset(rawMessage); - } catch (IOException e) { - throw new RuntimeException("Failed to adjust offset.", e); + ParsedMessage parsedMessage = null; + try { + parsedMessage = mMessageParser.parse(rawMessage); + final double DECAY = 0.999; + mUnparsableMessages *= DECAY; + } catch (Exception e) { + mUnparsableMessages++; + final double MAX_UNPARSABLE_MESSAGES = 1000.; + if (mUnparsableMessages > MAX_UNPARSABLE_MESSAGES) { + throw new RuntimeException("Failed to parse message " + rawMessage, e); } - ParsedMessage parsedMessage = null; + LOG.warn("Failed to parse message " + rawMessage, e); + } + + if (parsedMessage != null) { try { - parsedMessage = mMessageParser.parse(rawMessage); - final double DECAY = 0.999; - mUnparsableMessages *= DECAY; + mMessageWriter.write(parsedMessage); } 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 (Exception 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; } /** From ecf187fa3b33981f1daf1350b838deef4c34fabd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mathias=20S=C3=B6derberg?= Date: Fri, 31 Oct 2014 20:20:10 +0100 Subject: [PATCH 023/330] Add basic Travis CI configuration --- .travis.yml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 000000000..b41b756f0 --- /dev/null +++ b/.travis.yml @@ -0,0 +1,6 @@ +language: java +jdk: + - oraclejdk8 + - oraclejdk7 + - openjdk6 + - openjdk7 \ No newline at end of file From 06a964e865612abc00cd3e438d70f7a1ad663517 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Tue, 18 Nov 2014 00:54:55 -0800 Subject: [PATCH 024/330] add end to end tests for delimited text file reader/writer --- .../impl/DelimitedTextFileReaderWriter.java | 19 +++--- .../io/impl/SequenceFileReaderWriter.java | 7 +- .../main/TestLogMessageProducerMain.java | 11 +++- .../secor/tools/LogFileVerifier.java | 59 +++++++++-------- .../secor/tools/TestLogMessageProducer.java | 25 ++++++-- src/main/scripts/run_tests.sh | 64 ++++++++++++------- .../secor/io/FileReaderWriterTest.java | 62 +++++++++++------- 7 files changed, 156 insertions(+), 91 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java index 9dcfdaa57..c1b564674 100644 --- a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java @@ -20,18 +20,19 @@ import java.io.BufferedOutputStream; import java.io.ByteArrayOutputStream; import java.io.EOFException; -import java.io.File; -import java.io.FileInputStream; import java.io.FileNotFoundException; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.compress.CompressionCodec; import com.google.common.io.CountingOutputStream; import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.io.FileReaderWriter; import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.FileUtil; /** * @@ -56,11 +57,10 @@ public DelimitedTextFileReaderWriter(LogFilePath path, CompressionCodec codec, FileReaderWriter.Type type) throws FileNotFoundException, IOException { - File logFile = new File(path.getLogFilePath()); - logFile.getParentFile().mkdirs(); + Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); if (type == FileReaderWriter.Type.Reader) { - InputStream inputStream = new FileInputStream(new File( - path.getLogFilePath())); + InputStream inputStream = fs.open(fsPath); this.mReader = (codec == null) ? new BufferedInputStream(inputStream) : new BufferedInputStream( codec.createInputStream(inputStream)); @@ -68,8 +68,7 @@ public DelimitedTextFileReaderWriter(LogFilePath path, this.mCountingStream = null; this.mWriter = null; } else if (type == FileReaderWriter.Type.Writer) { - this.mCountingStream = new CountingOutputStream( - new FileOutputStream(logFile)); + this.mCountingStream = new CountingOutputStream(fs.create(fsPath)); this.mWriter = (codec == null) ? new BufferedOutputStream( this.mCountingStream) : new BufferedOutputStream( codec.createOutputStream(this.mCountingStream)); @@ -121,4 +120,4 @@ public KeyValue next() throws IOException { return new KeyValue(this.mOffset++, messageBuffer.toByteArray()); } -} +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java index fa5d67d8f..68300fa76 100644 --- a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java @@ -29,10 +29,11 @@ import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.io.FileReaderWriter; import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.FileUtil; /** * - * Sequence file writer implementation + * Sequence file reader writer implementation * * @author Praveen Murugesan (praveen@uber.com) * @@ -48,8 +49,8 @@ public class SequenceFileReaderWriter implements FileReaderWriter { public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec, FileReaderWriter.Type type) throws Exception { Configuration config = new Configuration(); - FileSystem fs = FileSystem.get(config); Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); if (type == FileReaderWriter.Type.Reader) { this.mReader = new SequenceFile.Reader(fs, fsPath, config); @@ -105,4 +106,4 @@ public KeyValue next() throws IOException { } } -} +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java b/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java index 192dac091..d05a08f04 100644 --- a/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java +++ b/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java @@ -57,6 +57,12 @@ private static CommandLine parseArgs(String[] args) throws ParseException { .withArgName("") .withType(Number.class) .create("p")); + options.addOption(OptionBuilder.withLongOpt("type") + .withDescription("type of producer") + .hasArg() + .withArgName("") + .withType(String.class) + .create("type")); CommandLineParser parser = new GnuParser(); return parser.parse(options, args); @@ -68,8 +74,9 @@ 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 type = commandLine.getOptionValue("type"); for (int i = 0; i < producers; ++i) { - TestLogMessageProducer producer = new TestLogMessageProducer(topic, messages); + TestLogMessageProducer producer = new TestLogMessageProducer(topic, messages, type); producer.start(); } } catch (Throwable t) { @@ -77,4 +84,4 @@ public static void main(String[] args) { System.exit(1); } } -} +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java index 681374d1b..58e4e7b97 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.FileReaderWriter; +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.*; @@ -107,15 +107,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(); + FileReaderWriter reader = createFileReaderWriter(logFilePath); int result = 0; - while (reader.next(key, value)) { + while (reader.next() != null) { result++; } reader.close(); @@ -162,16 +156,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 " + + FileReaderWriter reader = createFileReaderWriter(logFilePath); + KeyValue record; + while ((record = reader.next()) != null) { + if (!offsets.add(record.getKey())) { + throw new RuntimeException("duplicate key " + record.getKey() + " found in file " + logFilePath.getLogFilePath()); } } @@ -205,4 +194,24 @@ public void verifySequences(long fromOffset, long toOffset) throws Exception { } } } -} + + /** + * Helper to create a file reader writer from config + * + * @param logFilePath + * @return + * @throws Exception + */ + private FileReaderWriter createFileReaderWriter(LogFilePath logFilePath) throws Exception { + CompressionCodec codec = null; + if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + codec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); + } + FileReaderWriter fileReader = (FileReaderWriter) ReflectionUtil.createFileReaderWriter( + mConfig.getFileReaderWriter(), + logFilePath, + codec, + FileReaderWriter.Type.Reader); + return fileReader; + } +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java index e85143b3b..198c613fb 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,12 +37,14 @@ * @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; - public TestLogMessageProducer(String topic, int numMessages) { + public TestLogMessageProducer(String topic, int numMessages, String type) { mTopic = topic; mNumMessages = numMessages; + mType = type; } public void run() { @@ -52,7 +58,16 @@ 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"); + } + + TSerializer serializer = new TSerializer(protocol); for (int i = 0; i < mNumMessages; ++i) { TestMessage testMessage = new TestMessage(System.currentTimeMillis() * 1000000L + i, "some_value_" + i); @@ -73,4 +88,4 @@ public void run() { } producer.close(); } -} +} \ No newline at end of file diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 9af93ac3f..72b2dee04 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -40,8 +40,16 @@ PARENT_DIR=/tmp/secor_dev LOGS_DIR=${PARENT_DIR}/logs S3_LOGS_DIR=s3://pinterest-dev/secor_dev MESSAGES=1000 +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 +declare -A READER_WRITERS +READER_WRITERS[json]=com.pinterest.secor.io.impl.DelimitedTextFileReaderWriter +READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriter # 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. @@ -83,24 +91,28 @@ stop_kafka_server() { start_secor() { run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + -Dconfig=secor.dev.backup.properties ${ADDITIONAL_OPTS} -cp secor-0.1-SNAPSHOT.jar:lib/* \ com.pinterest.secor.main.ConsumerMain > ${LOGS_DIR}/secor_backup.log 2>&1 &" - run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.partition.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ - com.pinterest.secor.main.ConsumerMain > ${LOGS_DIR}/secor_partition.log 2>&1 &" + if [ "${MESSAGE_TYPE}" = "binary" ]; then + run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.dev.partition.properties ${ADDITIONAL_OPTS} -cp secor-0.1-SNAPSHOT.jar:lib/* \ + 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 \ + -Dconfig=secor.dev.backup.properties ${ADDITIONAL_OPTS} -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 &" + if [ "${MESSAGE_TYPE}" = "binary" ]; then + run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.dev.partition.properties ${ADDITIONAL_OPTS} -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 &" + fi } stop_secor() { @@ -116,19 +128,21 @@ create_topic() { 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 > \ + com.pinterest.secor.main.TestLogMessageProducerMain -t test -m $1 -p 1 -type ${MESSAGE_TYPE} > \ ${LOGS_DIR}/test_log_message_producer.log 2>&1" } verify() { run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + -Dconfig=secor.dev.backup.properties ${ADDITIONAL_OPTS} -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" + if [ "${MESSAGE_TYPE}" = "binary" ]; then + run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.dev.partition.properties ${ADDITIONAL_OPTS} -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" + fi } set_offsets_in_zookeeper() { @@ -205,7 +219,7 @@ start_from_non_zero_offset_test() { } # 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 "running move_offset_back_test" initialize @@ -218,7 +232,7 @@ move_offset_back_test() { 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. + # 4 because we skipped 2 messages per topic partition and there are 2 partitions per topic. verify $((${MESSAGES}-4)) stop_all @@ -242,8 +256,12 @@ post_and_verify_compressed_test() { } - -post_and_verify_test -start_from_non_zero_offset_test -move_offset_back_test -post_and_verify_compressed_test +for key in ${!READER_WRITERS[@]}; do + MESSAGE_TYPE=${key} + ADDITIONAL_OPTS=-Dsecor.file.reader.writer=${READER_WRITERS[${key}]} + echo "Running tests for Message Type: ${MESSAGE_TYPE} and ReaderWriter: ${READER_WRITERS[${key}]}" + post_and_verify_test + start_from_non_zero_offset_test + move_offset_back_test + post_and_verify_compressed_test +done diff --git a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java index ad01552f7..4fd32e39f 100644 --- a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java +++ b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java @@ -20,9 +20,12 @@ import java.io.FileOutputStream; import java.io.InputStream; import java.io.OutputStream; +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; @@ -54,7 +57,7 @@ @RunWith(PowerMockRunner.class) @PrepareForTest({ FileSystem.class, DelimitedTextFileReaderWriter.class, SequenceFile.class, SequenceFileReaderWriter.class, GzipCodec.class, - FileInputStream.class, FileOutputStream.class}) + FileInputStream.class, FileOutputStream.class }) public class FileReaderWriterTest extends TestCase { private static final String PATH = "/some_parent_dir/some_topic/some_partition/some_other_partition/" @@ -80,26 +83,33 @@ private void setupSequenceFileReaderConfig() { mConfig = new SecorConfig(properties); } - private void setupGzipFileReaderConfig() { + private void setupDelimitedTextFileReaderWriterConfig() { PropertiesConfiguration properties = new PropertiesConfiguration(); properties.addProperty("secor.file.reader.writer", "com.pinterest.secor.io.impl.DelimitedTextFileReaderWriter"); mConfig = new SecorConfig(properties); } - private void mockGzipFileReaderWriter() throws Exception { + private void mockDelimitedTextFileReaderWriter() 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 = new Path(PATH_GZ); + GzipCodec codec = PowerMockito.mock(GzipCodec.class); PowerMockito.whenNew(GzipCodec.class).withNoArguments() .thenReturn(codec); - FileInputStream fileInputStream = Mockito.mock(FileInputStream.class); - FileOutputStream fileOutputStream = Mockito.mock(FileOutputStream.class); - - PowerMockito.whenNew(FileInputStream.class).withAnyArguments() - .thenReturn(fileInputStream); + FSDataInputStream fileInputStream = Mockito + .mock(FSDataInputStream.class); + FSDataOutputStream fileOutputStream = Mockito + .mock(FSDataOutputStream.class); - PowerMockito.whenNew(FileOutputStream.class).withAnyArguments() - .thenReturn(fileOutputStream); + Mockito.when(fs.open(fsPath)).thenReturn(fileInputStream); + Mockito.when(fs.create(fsPath)).thenReturn(fileOutputStream); CompressionInputStream inputStream = Mockito .mock(CompressionInputStream.class); @@ -116,10 +126,12 @@ private void mockSequenceFileReaderWriter(boolean isCompressed) throws Exception { PowerMockito.mockStatic(FileSystem.class); FileSystem fs = Mockito.mock(FileSystem.class); - Mockito.when(FileSystem.get(Mockito.any(Configuration.class))) - .thenReturn(fs); + 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 @@ -170,7 +182,8 @@ public void testSequenceFileReader() throws Exception { // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); - FileSystem.get(Mockito.any(Configuration.class)); + FileSystem + .get(Mockito.any(URI.class), Mockito.any(Configuration.class)); } public void testSequenceFileWriter() throws Exception { @@ -183,7 +196,8 @@ public void testSequenceFileWriter() throws Exception { // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); - FileSystem.get(Mockito.any(Configuration.class)); + FileSystem + .get(Mockito.any(URI.class), Mockito.any(Configuration.class)); assert writer.getLength() == 123L; @@ -194,24 +208,26 @@ public void testSequenceFileWriter() throws Exception { // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); - FileSystem.get(Mockito.any(Configuration.class)); + FileSystem + .get(Mockito.any(URI.class), Mockito.any(Configuration.class)); assert writer.getLength() == 12L; } - public void testGzipFileWriter() throws Exception { - setupGzipFileReaderConfig(); - mockGzipFileReaderWriter(); + public void testDelimitedTextFileWriter() throws Exception { + setupDelimitedTextFileReaderWriterConfig(); + mockDelimitedTextFileReaderWriter(); FileReaderWriter writer = (FileReaderWriter) ReflectionUtil .createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Writer); + mLogFilePathGz, new GzipCodec(), + FileReaderWriter.Type.Writer); assert writer.getLength() == 0L; } - public void testGzipFileReader() throws Exception { - setupGzipFileReaderConfig(); - mockGzipFileReaderWriter(); + public void testDelimitedTextFileReader() throws Exception { + setupDelimitedTextFileReaderWriterConfig(); + mockDelimitedTextFileReaderWriter(); ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Reader); } -} +} \ No newline at end of file From 646a5882edfd427d2e5734ea40c85f323506263f Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Tue, 18 Nov 2014 10:24:30 -0800 Subject: [PATCH 025/330] enhanced file reader writer tests --- .../main/TestLogMessageProducerMain.java | 2 +- .../secor/tools/TestLogMessageProducer.java | 2 +- .../secor/io/FileReaderWriterTest.java | 30 +++++++++++++++---- 3 files changed, 27 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java b/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java index d05a08f04..6e768c3df 100644 --- a/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java +++ b/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java @@ -58,7 +58,7 @@ private static CommandLine parseArgs(String[] args) throws ParseException { .withType(Number.class) .create("p")); options.addOption(OptionBuilder.withLongOpt("type") - .withDescription("type of producer") + .withDescription("type of producer - [json, binary]") .hasArg() .withArgName("") .withType(String.class) diff --git a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java index 198c613fb..fee7070cd 100644 --- a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java +++ b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java @@ -64,7 +64,7 @@ public void run() { } else if (mType.equals("binary")) { protocol = new TBinaryProtocol.Factory(); } else { - throw new RuntimeException("Undefined message encoding type"); + throw new RuntimeException("Undefined message encoding type: " + mType); } TSerializer serializer = new TSerializer(protocol); diff --git a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java index 4fd32e39f..3e378331d 100644 --- a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java +++ b/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java @@ -90,14 +90,14 @@ private void setupDelimitedTextFileReaderWriterConfig() { mConfig = new SecorConfig(properties); } - private void mockDelimitedTextFileReaderWriter() throws Exception { + private void mockDelimitedTextFileReaderWriter(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 = new Path(PATH_GZ); + Path fsPath = (!isCompressed) ? new Path(PATH) : new Path(PATH_GZ); GzipCodec codec = PowerMockito.mock(GzipCodec.class); PowerMockito.whenNew(GzipCodec.class).withNoArguments() @@ -180,6 +180,14 @@ public void testSequenceFileReader() throws Exception { ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), mLogFilePath, null, FileReaderWriter.Type.Reader); + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem.get(Mockito.any(URI.class), Mockito.any(Configuration.class)); + + mockSequenceFileReaderWriter(true); + ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), + mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Reader); + // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); FileSystem @@ -203,7 +211,7 @@ public void testSequenceFileWriter() throws Exception { mockSequenceFileReaderWriter(true); - ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), + writer = (FileReaderWriter) ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Writer); // Verify that the method has been called exactly once (the default). @@ -216,8 +224,15 @@ public void testSequenceFileWriter() throws Exception { public void testDelimitedTextFileWriter() throws Exception { setupDelimitedTextFileReaderWriterConfig(); - mockDelimitedTextFileReaderWriter(); + mockDelimitedTextFileReaderWriter(false); FileReaderWriter writer = (FileReaderWriter) ReflectionUtil + .createFileReaderWriter(mConfig.getFileReaderWriter(), + mLogFilePath, null, + FileReaderWriter.Type.Writer); + assert writer.getLength() == 0L; + + mockDelimitedTextFileReaderWriter(true); + writer = (FileReaderWriter) ReflectionUtil .createFileReaderWriter(mConfig.getFileReaderWriter(), mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Writer); @@ -226,7 +241,12 @@ mLogFilePathGz, new GzipCodec(), public void testDelimitedTextFileReader() throws Exception { setupDelimitedTextFileReaderWriterConfig(); - mockDelimitedTextFileReaderWriter(); + + mockDelimitedTextFileReaderWriter(false); + ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), + mLogFilePath, null, FileReaderWriter.Type.Reader); + + mockDelimitedTextFileReaderWriter(true); ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Reader); } From 0e2fa9c23822d88b9929321a11cf9069ee48930d Mon Sep 17 00:00:00 2001 From: yongsheng Date: Fri, 21 Nov 2014 09:55:34 -0800 Subject: [PATCH 026/330] use java7 to run secor processes by default --- src/main/scripts/run_consumer.sh | 17 ++++++++++++-- src/main/scripts/run_kafka_class.sh | 8 ++++++- src/main/scripts/run_tests.sh | 27 +++++++++++++++++------ src/main/scripts/run_zookeeper_command.sh | 15 ++++++++++++- 4 files changed, 56 insertions(+), 11 deletions(-) diff --git a/src/main/scripts/run_consumer.sh b/src/main/scripts/run_consumer.sh index 834cc71f5..6ef539020 100755 --- a/src/main/scripts/run_consumer.sh +++ b/src/main/scripts/run_consumer.sh @@ -19,12 +19,25 @@ mkdir -p /mnt/secor_data/logs +# 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 + 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..516f48664 100755 --- a/src/main/scripts/run_kafka_class.sh +++ b/src/main/scripts/run_kafka_class.sh @@ -51,7 +51,13 @@ KAFKA_OPTS="" # Which java to use if [ -z "${JAVA_HOME}" ]; then - JAVA="java" + # 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 diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 72b2dee04..593f9203a 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -56,6 +56,19 @@ READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriter WAIT_TIME=120 base_dir=$(dirname $0) +# 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 + run_command() { echo "running $@" eval "$@" @@ -90,24 +103,24 @@ stop_kafka_server() { } start_secor() { - run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ -Dconfig=secor.dev.backup.properties ${ADDITIONAL_OPTS} -cp secor-0.1-SNAPSHOT.jar:lib/* \ com.pinterest.secor.main.ConsumerMain > ${LOGS_DIR}/secor_backup.log 2>&1 &" if [ "${MESSAGE_TYPE}" = "binary" ]; then - run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ -Dconfig=secor.dev.partition.properties ${ADDITIONAL_OPTS} -cp secor-0.1-SNAPSHOT.jar:lib/* \ 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 \ + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ -Dconfig=secor.dev.backup.properties ${ADDITIONAL_OPTS} -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 &" if [ "${MESSAGE_TYPE}" = "binary" ]; then - run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ -Dconfig=secor.dev.partition.properties ${ADDITIONAL_OPTS} -Djava.library.path=$HADOOP_NATIVE_LIB_PATH \ -Dsecor.compression.codec=org.apache.hadoop.io.compress.GzipCodec \ -cp secor-0.1-SNAPSHOT.jar:lib/* \ @@ -126,19 +139,19 @@ create_topic() { } post_messages() { - run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + 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 -type ${MESSAGE_TYPE} > \ ${LOGS_DIR}/test_log_message_producer.log 2>&1" } verify() { - run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ -Dconfig=secor.dev.backup.properties ${ADDITIONAL_OPTS} -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" if [ "${MESSAGE_TYPE}" = "binary" ]; then - run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ -Dconfig=secor.dev.partition.properties ${ADDITIONAL_OPTS} -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" diff --git a/src/main/scripts/run_zookeeper_command.sh b/src/main/scripts/run_zookeeper_command.sh index 8c0128e5b..e5e49c49f 100755 --- a/src/main/scripts/run_zookeeper_command.sh +++ b/src/main/scripts/run_zookeeper_command.sh @@ -22,4 +22,17 @@ if [ $# -lt 3 ]; then exit 1 fi -java -ea -cp "secor-0.1-SNAPSHOT.jar:lib/*" org.apache.zookeeper.ZooKeeperMain -server $@ +# 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 + +${JAVA} -ea -cp "secor-0.1-SNAPSHOT.jar:lib/*" org.apache.zookeeper.ZooKeeperMain -server $@ From c3e95ab6e70a0074f32d7851ca56b87ed46d15d8 Mon Sep 17 00:00:00 2001 From: yongsheng Date: Fri, 21 Nov 2014 16:08:16 -0800 Subject: [PATCH 027/330] refactor to share the logic to determine which java to use --- src/main/scripts/run_common.sh | 14 ++++++++++++++ src/main/scripts/run_consumer.sh | 14 ++------------ src/main/scripts/run_kafka_class.sh | 14 ++------------ src/main/scripts/run_tests.sh | 14 ++------------ src/main/scripts/run_zookeeper_command.sh | 14 ++------------ 5 files changed, 22 insertions(+), 48 deletions(-) create mode 100755 src/main/scripts/run_common.sh diff --git a/src/main/scripts/run_common.sh b/src/main/scripts/run_common.sh new file mode 100755 index 000000000..500a44eef --- /dev/null +++ b/src/main/scripts/run_common.sh @@ -0,0 +1,14 @@ +#!/bin/sh + +# 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 diff --git a/src/main/scripts/run_consumer.sh b/src/main/scripts/run_consumer.sh index 6ef539020..242737520 100755 --- a/src/main/scripts/run_consumer.sh +++ b/src/main/scripts/run_consumer.sh @@ -19,18 +19,8 @@ mkdir -p /mnt/secor_data/logs -# 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 +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 \ diff --git a/src/main/scripts/run_kafka_class.sh b/src/main/scripts/run_kafka_class.sh index 516f48664..bc41ec211 100755 --- a/src/main/scripts/run_kafka_class.sh +++ b/src/main/scripts/run_kafka_class.sh @@ -49,18 +49,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 - # 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 +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 593f9203a..46c1d40c8 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -56,18 +56,8 @@ READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriter WAIT_TIME=120 base_dir=$(dirname $0) -# 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 +CURR_DIR=`dirname $0` +source ${CURR_DIR}/run_common.sh run_command() { echo "running $@" diff --git a/src/main/scripts/run_zookeeper_command.sh b/src/main/scripts/run_zookeeper_command.sh index e5e49c49f..5e9e9450e 100755 --- a/src/main/scripts/run_zookeeper_command.sh +++ b/src/main/scripts/run_zookeeper_command.sh @@ -22,17 +22,7 @@ if [ $# -lt 3 ]; then exit 1 fi -# 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 +CURR_DIR=`dirname $0` +source ${CURR_DIR}/run_common.sh ${JAVA} -ea -cp "secor-0.1-SNAPSHOT.jar:lib/*" org.apache.zookeeper.ZooKeeperMain -server $@ From e8fddf2a7bb8f56a01f0fa341bd7a65e71b820e2 Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Thu, 13 Nov 2014 21:06:47 -0800 Subject: [PATCH 028/330] MessagePack parser with unit tests. Want to do some more manual testing and add a couple more tests. --- README.md | 3 + pom.xml | 5 + src/main/config/secor.common.properties | 2 +- .../secor/parser/MessagePackParser.java | 59 ++++++++ .../secor/parser/MessagePackParserTest.java | 132 ++++++++++++++++++ 5 files changed, 200 insertions(+), 1 deletion(-) create mode 100644 src/main/java/com/pinterest/secor/parser/MessagePackParser.java create mode 100644 src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java diff --git a/README.md b/README.md index 4ef260ce5..4fe705c61 100644 --- a/README.md +++ b/README.md @@ -53,6 +53,9 @@ One of the convenience features of Secor is the ability to group messages and sa - **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. +- **msgpack 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. + + 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. ## Tools diff --git a/pom.xml b/pom.xml index 4053c7af6..7784d65c2 100644 --- a/pom.xml +++ b/pom.xml @@ -151,6 +151,11 @@ 1.5.2 test + + org.msgpack + msgpack + 0.6.11 + diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 26d82890e..3983cedcd 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -90,7 +90,7 @@ tsdb.hostport= # Regex of topics that are not exported to TSDB. tsdb.blacklist.topics= -# Name of field that contains timestamp for JSON or Thrift message parser. (1405970352123) +# Name of field that contains timestamp for JSON, MessagePack, or Thrift message parser. (1405970352123) message.timestamp.name=timestamp # Name of field that contains a timestamp, as a date Format, for JSON. (2014-08-07, Jul 23 02:16:57 2005, etc...) 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..281b5e83c --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/MessagePackParser.java @@ -0,0 +1,59 @@ +/** + * 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.msgpack.MessagePack; +import org.msgpack.MessageTypeException; +import org.msgpack.type.MapValue; +import org.msgpack.type.RawValue; +import org.msgpack.type.Value; +import org.msgpack.type.ValueFactory; + +/** + * 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 MessagePack mMessagePack = new MessagePack(); + private RawValue mTimestampField; + + public MessagePackParser(SecorConfig config) { + super(config); + String timestampName = mConfig.getMessageTimestampName(); + mTimestampField = ValueFactory.createRawValue(timestampName); + } + + @Override + public long extractTimestampMillis(Message message) throws Exception { + MapValue msgMap = mMessagePack.read(message.getPayload()).asMapValue(); + Value timestampValue = msgMap.get(mTimestampField); + + if (timestampValue.isIntegerValue()) { + return toMillis(timestampValue.asIntegerValue().getLong()); + } else if (timestampValue.isFloatValue()) { + return toMillis(timestampValue.asFloatValue().longValue()); + } else { + String timestampString = timestampValue.asRawValue().getString(); + return toMillis(Long.parseLong(timestampString)); + } + } +} 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..1cb06ab47 --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -0,0 +1,132 @@ +/** + * 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.msgpack.MessagePack; +import org.msgpack.MessageTypeException; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.HashMap; + +@RunWith(PowerMockRunner.class) +public class MessagePackParserTest extends TestCase { + + private MessagePackParser mMessagePackParser; + private MessagePack mMessagePack; + private Message mMessageWithSecondsTimestamp; + private Message mMessageWithMillisTimestamp; + private Message mMessageWithMillisFloatTimestamp; + private Message mMessageWithMillisStringTimestamp; + + @Override + public void setUp() throws Exception { + SecorConfig mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("ts"); + mMessagePackParser = new MessagePackParser(mConfig); + + mMessagePack = new MessagePack(); + + HashMap mapWithSecondTimestamp = new HashMap(); + mapWithSecondTimestamp.put("ts", 1405970352); + mMessageWithSecondsTimestamp = new Message("test", 0, 0, + mMessagePack.write(mapWithSecondTimestamp)); + + 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, + mMessagePack.write(mapWithMillisTimestamp)); + + 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, + mMessagePack.write(mapWithMillisFloatTimestamp)); + + 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, + mMessagePack.write(mapWithMillisStringTimestamp)); + + } + + @Test + public void testExtractTimestampMillis() throws Exception { + assertEquals(1405970352000l, mMessagePackParser.extractTimestampMillis( + mMessageWithSecondsTimestamp)); + assertEquals(1405970352123l, mMessagePackParser.extractTimestampMillis( + mMessageWithMillisTimestamp)); + assertEquals(1405970352123l, mMessagePackParser.extractTimestampMillis( + mMessageWithMillisFloatTimestamp)); + assertEquals(1405970352123l, mMessagePackParser.extractTimestampMillis( + 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, + mMessagePack.write(mapWithoutTimestamp)); + mMessagePackParser.extractTimestampMillis(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, + mMessagePack.write(mapWitUnsupportedFormatTimestamp)); + mMessagePackParser.extractTimestampMillis(nMessageWithUnsupportedFormatTimestamp); + } + + @Test(expected=MessageTypeException.class) + public void testNullTimestamp() throws Exception { + HashMap mapWitNullTimestamp = new HashMap(); + mapWitNullTimestamp.put("ts", null); + Message nMessageWithNullTimestamp = new Message("test", 0, 0, + mMessagePack.write(mapWitNullTimestamp)); + mMessagePackParser.extractTimestampMillis(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]); + } +} From e1cddaedf5972db923da6016645429eda3fe3130 Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Mon, 24 Nov 2014 10:56:47 -0800 Subject: [PATCH 029/330] Update contributors list --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 4fe705c61..3482ed7aa 100644 --- a/README.md +++ b/README.md @@ -106,6 +106,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Lucas Zago](https://github.com/zago) * [James Green] (https://github.com/jfgreen) * [Praveen Murugesan] (https://github.com/lefthandmagic) + * [Zack Dever](https://github.com/zackdever) ## Help From af25b2bd0cc9a6e3f8ab30a87adaf1364ef07bef Mon Sep 17 00:00:00 2001 From: yongsheng Date: Tue, 25 Nov 2014 13:13:03 -0800 Subject: [PATCH 030/330] make sure that we use bash shell --- src/main/scripts/run_common.sh | 2 +- src/main/scripts/run_consumer.sh | 2 +- src/main/scripts/run_tests.sh | 3 +-- src/main/scripts/run_zookeeper_command.sh | 2 +- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/main/scripts/run_common.sh b/src/main/scripts/run_common.sh index 500a44eef..46715a060 100755 --- a/src/main/scripts/run_common.sh +++ b/src/main/scripts/run_common.sh @@ -1,4 +1,4 @@ -#!/bin/sh +#!/usr/bin/env bash # Which java to use if [ -z "${JAVA_HOME}" ]; then diff --git a/src/main/scripts/run_consumer.sh b/src/main/scripts/run_consumer.sh index 242737520..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 diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 46c1d40c8..d797416a8 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -56,8 +56,7 @@ READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriter WAIT_TIME=120 base_dir=$(dirname $0) -CURR_DIR=`dirname $0` -source ${CURR_DIR}/run_common.sh +source ${base_dir}/run_common.sh run_command() { echo "running $@" diff --git a/src/main/scripts/run_zookeeper_command.sh b/src/main/scripts/run_zookeeper_command.sh index 5e9e9450e..6dca38e93 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 From 681923b7c460868f7a146af1b10d4aa7a0adaf12 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Tue, 25 Nov 2014 23:04:22 -0800 Subject: [PATCH 031/330] pass in the codec to the verifier for delimited text reader --- src/main/scripts/run_tests.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 72b2dee04..6dc18aef9 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -249,6 +249,7 @@ post_and_verify_compressed_test() { post_messages ${MESSAGES} echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" sleep ${WAIT_TIME} + ADDITIONAL_OPTS="${ADDITIONAL_OPTS} -Dsecor.compression.codec=org.apache.hadoop.io.compress.GzipCodec" verify ${MESSAGES} stop_all From fae4fd25d5291c822133bdffd4c165960ae45699 Mon Sep 17 00:00:00 2001 From: Pawel Garbacki Date: Wed, 26 Nov 2014 15:00:24 -0800 Subject: [PATCH 032/330] Added a param to control rebalance backoff --- src/main/config/secor.common.properties | 3 +++ .../java/com/pinterest/secor/reader/MessageReader.java | 10 ++++++++-- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 3983cedcd..f26897951 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -38,6 +38,9 @@ kafka.consumer.timeout.ms=10000 # 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= diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index f7964c3d8..c24d9ef4b 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -110,10 +110,16 @@ private ConsumerConfig createConsumerConfig() throws UnknownHostException { props.put("auto.offset.reset", "smallest"); props.put("consumer.timeout.ms", Integer.toString(mConfig.getConsumerTimeoutMs())); props.put("consumer.id", IdUtil.getConsumerId()); - if (mConfig.getRebalanceMaxRetries() != null && !mConfig.getRebalanceMaxRetries().isEmpty()) { + if (mConfig.getRebalanceMaxRetries() != null && + !mConfig.getRebalanceMaxRetries().isEmpty()) { props.put("rebalance.max.retries", mConfig.getRebalanceMaxRetries()); } - if (mConfig.getSocketReceieveBufferBytes() != null && !mConfig.getSocketReceieveBufferBytes().isEmpty()) { + if (mConfig.getRebalanceBackoffMs() != null && + !mConfig.getRebalanceBackoffMs().isEmpty()) { + props.put("rebalance.backoff.ms", mConfig.getRebalanceBackoffMs()); + } + if (mConfig.getSocketReceieveBufferBytes() != null && + !mConfig.getSocketReceieveBufferBytes().isEmpty()) { props.put("socket.receive.buffer.bytes", mConfig.getSocketReceieveBufferBytes()); } if (mConfig.getFetchMessageMaxBytes() != null && !mConfig.getFetchMessageMaxBytes().isEmpty()) { From 25f9e7d942450989b64633215f53d4f3f0286b1c Mon Sep 17 00:00:00 2001 From: "Ernest W. Durbin III" Date: Thu, 27 Nov 2014 12:04:38 -0500 Subject: [PATCH 033/330] add missing method on SecorConfig class --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 479c4bf44..e0f1437fc 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -91,6 +91,10 @@ 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"); } From 98ec608b0ec2e20b06339778408ade9cb1cad699 Mon Sep 17 00:00:00 2001 From: Pawel Garbacki Date: Thu, 27 Nov 2014 16:53:52 -0800 Subject: [PATCH 034/330] Handle the case of empty partitions in the finalizer --- .../java/com/pinterest/secor/parser/PartitionFinalizer.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 458b9855d..873bdcbca 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -65,6 +65,12 @@ public PartitionFinalizer(SecorConfig config) throws Exception { private long getLastTimestampMillis(TopicPartition topicPartition) throws TException { Message message = mKafkaClient.getLastMessage(topicPartition); + if (message == null) { + // This will happen if no messages have been posted to the given topic partition. + LOG.error("No message found for topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition()); + return -1; + } return mThriftMessageParser.extractTimestampMillis(message); } From 9b48e492eff4a9c2fc162c553be9168dc749fd77 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Tue, 25 Nov 2014 08:19:50 -0800 Subject: [PATCH 035/330] ability to export stats using statsD for monitoring in graphite etc make sure that we use bash shell pass in the codec to the verifier for delimited text reader Added a param to control rebalance backoff add missing method on SecorConfig class Handle the case of empty partitions in the finalizer proper formatting for pom.xml --- README.md | 5 +- pom.xml | 5 ++ src/main/config/secor.common.properties | 6 +- .../pinterest/secor/common/SecorConfig.java | 8 +- .../secor/tools/ProgressMonitor.java | 77 +++++++++++++++++-- 5 files changed, 89 insertions(+), 12 deletions(-) diff --git a/README.md b/README.md index 3482ed7aa..1cab67501 100644 --- a/README.md +++ b/README.md @@ -9,7 +9,7 @@ Secor is a service persisting [Kafka] logs to [Amazon S3]. - **horizontal scalability**: scaling the system out to handle more load is as easy as starting extra Secor processes. Reducing the resource footprint can be achieved by killing any of the running Secor processes. Neither ramping up nor down has any impact on data consistency, - **output partitioning**: Secor parses incoming messages and puts them under partitioned s3 paths to enable direct import into systems like [Hive], - **configurable upload policies**: commit points controlling when data is persisted in S3 are configured through size-based and time-based policies (e.g., upload data when local buffer reaches size of 100MB and at least once per hour), - - **monitoring**: metrics tracking various performace properties are exposed through [Ostrich] and optionaly exported to [OpenTSDB], + - **monitoring**: metrics tracking various performace properties are exposed through [Ostrich] and optionaly exported to [OpenTSDB] / [statsD], - **customizability**: external log message parser may be loaded by updating the configuration, - **Qubole interface**: Secor connects to [Qubole] to add finalized output partitions to Hive tables. @@ -83,7 +83,7 @@ java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup. ``` ##### 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 @@ -120,4 +120,5 @@ If you have any questions or comments, you can reach us at [secor-users@googlegr [Ostrich]: https://github.com/twitter/ostrich [OpenTSDB]: http://opentsdb.net/ [Qubole]: http://www.qubole.com/ +[statsD]: https://github.com/etsy/statsd/ diff --git a/pom.xml b/pom.xml index 7784d65c2..ddaf0ff90 100644 --- a/pom.xml +++ b/pom.xml @@ -156,6 +156,11 @@ msgpack 0.6.11 + + com.timgroup + java-statsd-client + 3.0.2 + diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index f26897951..31cc132f7 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -91,7 +91,11 @@ qubole.api.token= tsdb.hostport= # Regex of topics that are not exported to TSDB. -tsdb.blacklist.topics= +monitoring.blacklist.topics= + +# 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= # Name of field that contains timestamp for JSON, MessagePack, or Thrift message parser. (1405970352123) message.timestamp.name=timestamp diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index e0f1437fc..107c5ddf8 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -187,8 +187,12 @@ public String getTsdbHostport() { return getString("tsdb.hostport"); } - public String getTsdbBlacklistTopics() { - return getString("tsdb.blacklist.topics"); + public String getStatsDHostPort() { + return getString("statsd.hostport"); + } + + public String getMonitoringBlacklistTopics() { + return getString("monitoring.blacklist.topics"); } public String getMessageTimestampName() { diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index f7ea27d0b..693433f7d 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -18,6 +18,7 @@ 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; @@ -26,9 +27,12 @@ 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; @@ -48,7 +52,10 @@ * @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; @@ -115,11 +122,41 @@ private void exportToTsdb(Stat stat) public void exportStats() throws Exception { List stats = getStats(); System.out.println(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 (mConfig.getStatsDHostPort() != null && !mConfig.getStatsDHostPort().isEmpty()) { + exportToStatsD(stats); + } + } + + /** + * Helper to publish stats to statsD client + */ + private void exportToStatsD(List stats) { + HostAndPort hostPort = HostAndPort.fromString(mConfig.getStatsDHostPort()); + + // group stats by kafka group + NonBlockingStatsDClient client = new NonBlockingStatsDClient(mConfig.getKafkaGroup(), + hostPort.getHostText(), hostPort.getPort()); + + for (Stat stat : stats) { + @SuppressWarnings("unchecked") + Map tags = (Map) stat.get(Stat.STAT_KEYS.TAGS.getName()); + String aspect = new StringBuilder((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(); + client.recordGaugeValue(aspect, (Long)stat.get(Stat.STAT_KEYS.VALUE.getName())); + } } private List getStats() throws Exception { @@ -127,7 +164,7 @@ private List getStats() throws Exception { 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); continue; @@ -158,8 +195,8 @@ private List getStats() throws Exception { long offsetLag = lastOffset - committedOffset; long timestampMillisLag = lastTimestampMillis - committedTimestampMillis; Map tags = ImmutableMap.of( - "topic", topic, - "partition", Integer.toString(partition) + Stat.STAT_KEYS.TOPIC.getName(), topic, + Stat.STAT_KEYS.PARTITION.getName(), Integer.toString(partition) ); long timestamp = System.currentTimeMillis() / 1000; @@ -185,14 +222,39 @@ private long getTimestamp(Message message) throws Exception { } } + /** + * + * 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"); + + 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( - "metric", metric, - "tags", tags, - "value", value, - "timestamp", timestamp + STAT_KEYS.METRIC.getName(), metric, + STAT_KEYS.TAGS.getName(), tags, + STAT_KEYS.VALUE.getName(), value, + STAT_KEYS.TIMESTAMP.getName(), timestamp )); } @@ -200,4 +262,5 @@ public Stat(Map map) { super(map); } } + } From fc971f9c75eec8bfdea0ef37e56d196009a91480 Mon Sep 17 00:00:00 2001 From: Praveen Murugesan Date: Mon, 1 Dec 2014 22:53:36 -0800 Subject: [PATCH 036/330] refactor end to end tests --- README.md | 13 +++++++++++-- src/main/scripts/run_tests.sh | 21 ++++----------------- 2 files changed, 15 insertions(+), 19 deletions(-) diff --git a/README.md b/README.md index 3482ed7aa..24ebd287c 100644 --- a/README.md +++ b/README.md @@ -58,6 +58,15 @@ One of the convenience features of Secor is the ability to group messages and sa 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. + +- **Delimited Text Files**: A new line delimited raw text file. + ## Tools Secor comes with a number of tools impelementing interactions with the environment. @@ -104,8 +113,8 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [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) + * [James Green](https://github.com/jfgreen) + * [Praveen Murugesan](https://github.com/lefthandmagic) * [Zack Dever](https://github.com/zackdever) ## Help diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index fcfe0bf6f..36acb0e74 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -102,21 +102,6 @@ start_secor() { fi } -start_secor_compressed() { - run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.backup.properties ${ADDITIONAL_OPTS} -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 &" - if [ "${MESSAGE_TYPE}" = "binary" ]; then - run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.partition.properties ${ADDITIONAL_OPTS} -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 &" - fi -} - stop_secor() { run_command "pkill -f 'com.pinterest.secor.main.ConsumerMain' | true" } @@ -246,12 +231,14 @@ post_and_verify_compressed_test() { echo "running post_and_verify_compressed_test" initialize - start_secor_compressed + # 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} echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" sleep ${WAIT_TIME} - ADDITIONAL_OPTS="${ADDITIONAL_OPTS} -Dsecor.compression.codec=org.apache.hadoop.io.compress.GzipCodec" verify ${MESSAGES} stop_all From 48d5d7e698fb84d14cce2b2804aa4044dc73e5ce Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Thu, 18 Dec 2014 16:17:55 -0800 Subject: [PATCH 037/330] use msgpack 0.7.x to use new jackson based api --- pom.xml | 4 +-- .../secor/parser/MessagePackParser.java | 35 +++++++++---------- .../secor/parser/MessagePackParserTest.java | 26 +++++++------- 3 files changed, 32 insertions(+), 33 deletions(-) diff --git a/pom.xml b/pom.xml index ddaf0ff90..67cde0e0b 100644 --- a/pom.xml +++ b/pom.xml @@ -153,8 +153,8 @@ org.msgpack - msgpack - 0.6.11 + jackson-dataformat-msgpack + 0.7.0-p2 com.timgroup diff --git a/src/main/java/com/pinterest/secor/parser/MessagePackParser.java b/src/main/java/com/pinterest/secor/parser/MessagePackParser.java index 281b5e83c..8baa4ffd4 100644 --- a/src/main/java/com/pinterest/secor/parser/MessagePackParser.java +++ b/src/main/java/com/pinterest/secor/parser/MessagePackParser.java @@ -16,14 +16,13 @@ */ 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.MessagePack; -import org.msgpack.MessageTypeException; -import org.msgpack.type.MapValue; -import org.msgpack.type.RawValue; -import org.msgpack.type.Value; -import org.msgpack.type.ValueFactory; +import org.msgpack.jackson.dataformat.MessagePackFactory; + +import java.util.HashMap; /** * MessagePack timestamped message parser. @@ -33,27 +32,27 @@ * @author Zack Dever (zack@rd.io) */ public class MessagePackParser extends TimestampedMessageParser { - private MessagePack mMessagePack = new MessagePack(); - private RawValue mTimestampField; + private ObjectMapper mMessagePackObjectMapper; + private TypeReference mTypeReference; public MessagePackParser(SecorConfig config) { super(config); - String timestampName = mConfig.getMessageTimestampName(); - mTimestampField = ValueFactory.createRawValue(timestampName); + mMessagePackObjectMapper = new ObjectMapper(new MessagePackFactory()); + mTypeReference = new TypeReference>(){}; } @Override public long extractTimestampMillis(Message message) throws Exception { - MapValue msgMap = mMessagePack.read(message.getPayload()).asMapValue(); - Value timestampValue = msgMap.get(mTimestampField); + HashMap msgHash = mMessagePackObjectMapper.readValue(message.getPayload(), + mTypeReference); + Object timestampValue = msgHash.get(mConfig.getMessageTimestampName()); - if (timestampValue.isIntegerValue()) { - return toMillis(timestampValue.asIntegerValue().getLong()); - } else if (timestampValue.isFloatValue()) { - return toMillis(timestampValue.asFloatValue().longValue()); + if (timestampValue instanceof Number) { + return toMillis(((Number) timestampValue).longValue()); + } else if (timestampValue instanceof String) { + return toMillis(Long.parseLong((String) timestampValue)); } else { - String timestampString = timestampValue.asRawValue().getString(); - return toMillis(Long.parseLong(timestampString)); + return toMillis((Long) timestampValue); } } } diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java index 1cb06ab47..0b52b6128 100644 --- a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -17,14 +17,14 @@ 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.MessagePack; -import org.msgpack.MessageTypeException; +import org.msgpack.jackson.dataformat.MessagePackFactory; import org.powermock.modules.junit4.PowerMockRunner; import java.util.HashMap; @@ -33,24 +33,23 @@ public class MessagePackParserTest extends TestCase { private MessagePackParser mMessagePackParser; - private MessagePack mMessagePack; private Message mMessageWithSecondsTimestamp; private Message mMessageWithMillisTimestamp; private Message mMessageWithMillisFloatTimestamp; private Message mMessageWithMillisStringTimestamp; + private ObjectMapper mObjectMapper; @Override public void setUp() throws Exception { SecorConfig mConfig = Mockito.mock(SecorConfig.class); Mockito.when(mConfig.getMessageTimestampName()).thenReturn("ts"); mMessagePackParser = new MessagePackParser(mConfig); - - mMessagePack = new MessagePack(); + mObjectMapper = new ObjectMapper(new MessagePackFactory()); HashMap mapWithSecondTimestamp = new HashMap(); mapWithSecondTimestamp.put("ts", 1405970352); mMessageWithSecondsTimestamp = new Message("test", 0, 0, - mMessagePack.write(mapWithSecondTimestamp)); + mObjectMapper.writeValueAsBytes(mapWithSecondTimestamp)); HashMap mapWithMillisTimestamp = new HashMap(); mapWithMillisTimestamp.put("ts", 1405970352123l); @@ -58,7 +57,8 @@ public void setUp() throws Exception { mapWithMillisTimestamp.put("email", "alice@example.com"); mapWithMillisTimestamp.put("age", 27); mMessageWithMillisTimestamp = new Message("test", 0, 0, - mMessagePack.write(mapWithMillisTimestamp)); + mObjectMapper.writeValueAsBytes(mapWithMillisTimestamp)); + HashMap mapWithMillisFloatTimestamp = new HashMap(); mapWithMillisFloatTimestamp.put("ts", 1405970352123.0); @@ -66,7 +66,7 @@ public void setUp() throws Exception { mapWithMillisFloatTimestamp.put("email", "bob@example.com"); mapWithMillisFloatTimestamp.put("age", 35); mMessageWithMillisFloatTimestamp = new Message("test", 0, 0, - mMessagePack.write(mapWithMillisFloatTimestamp)); + mObjectMapper.writeValueAsBytes(mapWithMillisFloatTimestamp)); HashMap mapWithMillisStringTimestamp = new HashMap(); mapWithMillisStringTimestamp.put("ts", "1405970352123"); @@ -74,7 +74,7 @@ public void setUp() throws Exception { mapWithMillisStringTimestamp.put("email", "charlie@example.com"); mapWithMillisStringTimestamp.put("age", 67); mMessageWithMillisStringTimestamp = new Message("test", 0, 0, - mMessagePack.write(mapWithMillisStringTimestamp)); + mObjectMapper.writeValueAsBytes(mapWithMillisStringTimestamp)); } @@ -95,7 +95,7 @@ public void testMissingTimestamp() throws Exception { HashMap mapWithoutTimestamp = new HashMap(); mapWithoutTimestamp.put("email", "mary@example.com"); Message nMessageWithoutTimestamp = new Message("test", 0, 0, - mMessagePack.write(mapWithoutTimestamp)); + mObjectMapper.writeValueAsBytes(mapWithoutTimestamp)); mMessagePackParser.extractTimestampMillis(nMessageWithoutTimestamp); } @@ -104,16 +104,16 @@ 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, - mMessagePack.write(mapWitUnsupportedFormatTimestamp)); + mObjectMapper.writeValueAsBytes(mapWitUnsupportedFormatTimestamp)); mMessagePackParser.extractTimestampMillis(nMessageWithUnsupportedFormatTimestamp); } - @Test(expected=MessageTypeException.class) + @Test(expected=NullPointerException.class) public void testNullTimestamp() throws Exception { HashMap mapWitNullTimestamp = new HashMap(); mapWitNullTimestamp.put("ts", null); Message nMessageWithNullTimestamp = new Message("test", 0, 0, - mMessagePack.write(mapWitNullTimestamp)); + mObjectMapper.writeValueAsBytes(mapWitNullTimestamp)); mMessagePackParser.extractTimestampMillis(nMessageWithNullTimestamp); } From fff647adc268032aed1145be31f2e135cc3d5d21 Mon Sep 17 00:00:00 2001 From: Ramki Venkatachalam Date: Tue, 23 Dec 2014 06:20:02 +0000 Subject: [PATCH 038/330] Updating script to report success/failure more clearly --- src/main/scripts/run_tests.sh | 34 +++++++++++++++++++++------------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 36acb0e74..6a8ef2c03 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -34,7 +34,7 @@ # Author: Pawel Garbacki (pawel@pinterest.com) -set -e +#set -e PARENT_DIR=/tmp/secor_dev LOGS_DIR=${PARENT_DIR}/logs @@ -120,16 +120,23 @@ post_messages() { } verify() { - run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.backup.properties ${ADDITIONAL_OPTS} -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" + RUNMODE_0="backup" if [ "${MESSAGE_TYPE}" = "binary" ]; then - run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.partition.properties ${ADDITIONAL_OPTS} -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" + RUNMODE_1="partition" fi + for RUNMODE in ${RUNMODE_0} ${RUNMODE_1}; do + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.dev.${RUNMODE}.properties ${ADDITIONAL_OPTS} -cp secor-0.1-SNAPSHOT.jar:lib/* \ + 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 + exit ${VERIFICATION_EXIT_CODE} + fi + done } set_offsets_in_zookeeper() { @@ -186,7 +193,7 @@ post_and_verify_test() { verify ${MESSAGES} stop_all - echo "post_and_verify_test succeeded" + echo -e "\e[1;42;97mpost_and_verify_test succeeded\e[0m" } # Adjust offsets so that Secor consumes only half of the messages. @@ -202,7 +209,7 @@ start_from_non_zero_offset_test() { verify $((${MESSAGES}/2)) 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 @@ -223,7 +230,7 @@ move_offset_back_test() { verify $((${MESSAGES}-4)) stop_all - echo "move_offset_back_test succeeded" + echo -e "\e[1;42;97mmove_offset_back_test succeeded\e[0m" } # Post some messages and verify that they are correctly processed and compressed. @@ -242,7 +249,7 @@ post_and_verify_compressed_test() { verify ${MESSAGES} stop_all - echo "post_and_verify_compressed_test succeeded" + echo -e "\e[1;42;97mpost_and_verify_compressed_test succeeded\e[0m" } @@ -255,3 +262,4 @@ for key in ${!READER_WRITERS[@]}; do move_offset_back_test post_and_verify_compressed_test done + From 9e10143665b0e7b2577d9e75a9482acec6b0ea75 Mon Sep 17 00:00:00 2001 From: Ramki Venkatachalam Date: Tue, 23 Dec 2014 16:44:39 +0000 Subject: [PATCH 039/330] removing commented set -e --- src/main/scripts/run_tests.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 6a8ef2c03..35fa269c3 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -34,8 +34,6 @@ # 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 From 3c3edc651cff41023be8ba235758a066d024259a Mon Sep 17 00:00:00 2001 From: Ramki Venkatachalam Date: Tue, 23 Dec 2014 10:42:21 -0800 Subject: [PATCH 040/330] Adding support for finalizing topics that have json messages Adding support for optional prefix for hive tables --- pom.xml | 11 +++++++++++ src/main/config/secor.common.properties | 5 +++++ .../pinterest/secor/common/SecorConfig.java | 9 +++++++++ .../secor/parser/PartitionFinalizer.java | 18 +++++++++--------- 4 files changed, 34 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 67cde0e0b..fc4560e3d 100644 --- a/pom.xml +++ b/pom.xml @@ -48,6 +48,11 @@ + + net.java.dev.jets3t + jets3t + 0.7.1 + org.apache.kafka kafka_2.10 @@ -102,6 +107,12 @@ org.apache.hadoop hadoop-core 1.2.1 + + + net.java.dev.jets3t + jets3t + + org.apache.thrift diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 31cc132f7..234887ae2 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -86,6 +86,11 @@ 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= + # Secor can export stats such as consumption lag (in seconds and offsets) per topic partition. # Leave empty to disable this functionality. tsdb.hostport= diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 107c5ddf8..0d72fe6e3 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -203,6 +203,10 @@ public String getMessageTimestampInputPattern() { return getString("message.timestamp.input.pattern"); } + public String getHivePrefix() { + return getString("secor.hive.prefix", ""); + } + public String getCompressionCodec() { return getString("secor.compression.codec"); } @@ -231,6 +235,11 @@ private String getString(String name) { return mProperties.getString(name); } + private String getString(String name, String defaultValue) { + checkProperty(name); + return mProperties.getString(name, defaultValue); + } + private int getInt(String name) { checkProperty(name); return mProperties.getInt(name); diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 873bdcbca..558d0fd41 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -20,9 +20,8 @@ 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; @@ -44,7 +43,7 @@ public class PartitionFinalizer { private SecorConfig mConfig; private ZookeeperConnector mZookeeperConnector; - private ThriftMessageParser mThriftMessageParser; + private TimestampedMessageParser mMessageParser; private KafkaClient mKafkaClient; private QuboleClient mQuboleClient; private String mFileExtension; @@ -53,7 +52,8 @@ 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 = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); @@ -63,7 +63,7 @@ public PartitionFinalizer(SecorConfig config) throws Exception { } } - private long getLastTimestampMillis(TopicPartition topicPartition) throws TException { + private long getLastTimestampMillis(TopicPartition topicPartition) throws Exception { Message message = mKafkaClient.getLastMessage(topicPartition); if (message == null) { // This will happen if no messages have been posted to the given topic partition. @@ -71,10 +71,10 @@ private long getLastTimestampMillis(TopicPartition topicPartition) throws TExcep topicPartition.getPartition()); return -1; } - return mThriftMessageParser.extractTimestampMillis(message); + return mMessageParser.extractTimestampMillis(message); } - private long getLastTimestampMillis(String topic) throws TException { + private long getLastTimestampMillis(String topic) throws Exception { final int numPartitions = mKafkaClient.getNumPartitions(topic); long max_timestamp = Long.MIN_VALUE; for (int partition = 0; partition < numPartitions; ++partition) { @@ -97,7 +97,7 @@ private long getCommittedTimestampMillis(TopicPartition topicPartition) throws E topicPartition.getPartition()); return -1; } - return mThriftMessageParser.extractTimestampMillis(message); + return mMessageParser.extractTimestampMillis(message); } private long getCommittedTimestampMillis(String topic) throws Exception { @@ -162,7 +162,7 @@ private void finalizePartitionsUpTo(String topic, Calendar calendar) throws IOEx return; } try { - mQuboleClient.addPartition(topic, "dt='" + partitionStr + "'"); + mQuboleClient.addPartition(mConfig.getHivePrefix() + topic, "dt='" + partitionStr + "'"); } catch (Exception e) { LOG.error("failed to finalize topic " + topic + " partition dt=" + partitionStr, e); From 87dfac3494ebf1a1e0f25d15cd0e678cc2c9c533 Mon Sep 17 00:00:00 2001 From: Ivan Balashov Date: Sat, 27 Dec 2014 20:08:15 +0300 Subject: [PATCH 041/330] removed thread local from TimestampedMessageParser --- .../secor/parser/TimestampedMessageParser.java | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index e0653ea74..74e4b0a02 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -25,18 +25,12 @@ public abstract class TimestampedMessageParser extends MessageParser { - private static final ThreadLocal mFormatter = new ThreadLocal(){ - @Override - protected SimpleDateFormat initialValue() - { - SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd"); - format.setTimeZone(TimeZone.getTimeZone("UTC")); - return format; - } - }; + private SimpleDateFormat mFormatter; public TimestampedMessageParser(SecorConfig config) { super(config); + mFormatter = new SimpleDateFormat("yyyy-MM-dd"); + mFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); } public abstract long extractTimestampMillis(final Message message) throws Exception; @@ -60,7 +54,7 @@ public String[] extractPartitions(Message message) throws Exception { // Date constructor takes milliseconds since epoch. long timestampMillis = extractTimestampMillis(message); Date date = new Date(timestampMillis); - String result[] = {"dt=" + mFormatter.get().format(date)}; + String result[] = {"dt=" + mFormatter.format(date)}; return result; } } From 72dde9c7c7946e075833e398621f5fa5aacbb8c1 Mon Sep 17 00:00:00 2001 From: Ramki Venkatachalam Date: Mon, 5 Jan 2015 12:56:44 -0800 Subject: [PATCH 042/330] Cleaning up code based on review - need to specific config ; no defaults --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 0d72fe6e3..3d5e6c1a8 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -204,7 +204,7 @@ public String getMessageTimestampInputPattern() { } public String getHivePrefix() { - return getString("secor.hive.prefix", ""); + return getString("secor.hive.prefix"); } public String getCompressionCodec() { @@ -235,11 +235,6 @@ private String getString(String name) { return mProperties.getString(name); } - private String getString(String name, String defaultValue) { - checkProperty(name); - return mProperties.getString(name, defaultValue); - } - private int getInt(String name) { checkProperty(name); return mProperties.getInt(name); From 9859b62e88894aef26839f7322e66709abdc8919 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fl=C3=A1vio=20Roberto=20Santos?= Date: Thu, 8 Jan 2015 12:27:44 -0200 Subject: [PATCH 043/330] Generalizing number conversion This approach works for integer, float/double and scientific notation (e.g., 12E+9). The throws declaration was removed from method extractTimestampMillis(). --- .../pinterest/secor/parser/JsonMessageParser.java | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java index e5eb91c94..183bf7aa3 100644 --- a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java @@ -31,23 +31,12 @@ public JsonMessageParser(SecorConfig config) { } @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()); 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()); } } return 0; From 1ccbcd6f87e742510e1e68ffcbf43c1cdca755d2 Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Fri, 9 Jan 2015 14:28:09 -0800 Subject: [PATCH 044/330] fix google groups link in README. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index f1be8a3f7..be448b151 100644 --- a/README.md +++ b/README.md @@ -119,7 +119,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l ## 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/ From e76d015d90f45c5366442c83929c35fee8c3fca8 Mon Sep 17 00:00:00 2001 From: Leo Woessner Date: Fri, 16 Jan 2015 11:51:29 -0700 Subject: [PATCH 045/330] using length of BytesWritable to decide how much of the byte array to display. --- src/main/java/com/pinterest/secor/tools/LogFilePrinter.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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)); } } } From 7ad22b1bbe50a51bc32989cd0d12fe4eb2fa400a Mon Sep 17 00:00:00 2001 From: Leo Woessner Date: Fri, 16 Jan 2015 15:28:16 -0700 Subject: [PATCH 046/330] added estezz to contributors --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index be448b151..3a8bc86e1 100644 --- a/README.md +++ b/README.md @@ -116,6 +116,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [James Green](https://github.com/jfgreen) * [Praveen Murugesan](https://github.com/lefthandmagic) * [Zack Dever](https://github.com/zackdever) + * [Leo Woessner](https://github.com/estezz) ## Help From 3f757409da60ad9323411305a20ab08c45f17a1b Mon Sep 17 00:00:00 2001 From: Leo Woessner Date: Fri, 16 Jan 2015 11:51:29 -0700 Subject: [PATCH 047/330] Introduced combined FileReaderWriterFactory for describing output formats, replaces FileReaderWriter for improved clarity. --- README.md | 1 + src/main/config/secor.common.properties | 2 +- .../pinterest/secor/common/FileRegistry.java | 18 ++- .../pinterest/secor/common/SecorConfig.java | 4 +- .../pinterest/secor/consumer/Consumer.java | 3 +- .../com/pinterest/secor/io/FileReader.java | 44 ++++++ .../secor/io/FileReaderWriterFactory.java | 57 ++++++++ ...{FileReaderWriter.java => FileWriter.java} | 59 +++------ .../impl/DelimitedTextFileReaderWriter.java | 123 ----------------- .../DelimitedTextFileReaderWriterFactory.java | 125 ++++++++++++++++++ .../io/impl/SequenceFileReaderWriter.java | 109 --------------- .../impl/SequenceFileReaderWriterFactory.java | 120 +++++++++++++++++ .../pinterest/secor/tools/LogFilePrinter.java | 4 +- .../secor/tools/LogFileVerifier.java | 16 +-- .../pinterest/secor/uploader/Uploader.java | 17 +-- .../pinterest/secor/util/ReflectionUtil.java | 100 ++++++++++---- .../pinterest/secor/writer/MessageWriter.java | 4 +- src/main/scripts/run_tests.sh | 8 +- .../secor/common/FileRegistryTest.java | 38 +++--- ....java => FileReaderWriterFactoryTest.java} | 105 +++++++-------- .../SequenceFileReaderWriterFactoryTest.java | 64 +++++++++ .../secor/uploader/UploaderTest.java | 15 ++- .../secor/util/ReflectionUtilTest.java | 67 ++++++++++ 23 files changed, 683 insertions(+), 420 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/io/FileReader.java create mode 100644 src/main/java/com/pinterest/secor/io/FileReaderWriterFactory.java rename src/main/java/com/pinterest/secor/io/{FileReaderWriter.java => FileWriter.java} (53%) delete mode 100644 src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java create mode 100644 src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java delete mode 100644 src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java create mode 100644 src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java rename src/test/java/com/pinterest/secor/io/{FileReaderWriterTest.java => FileReaderWriterFactoryTest.java} (70%) create mode 100644 src/test/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactoryTest.java create mode 100644 src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java diff --git a/README.md b/README.md index be448b151..3a8bc86e1 100644 --- a/README.md +++ b/README.md @@ -116,6 +116,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [James Green](https://github.com/jfgreen) * [Praveen Murugesan](https://github.com/lefthandmagic) * [Zack Dever](https://github.com/zackdever) + * [Leo Woessner](https://github.com/estezz) ## Help diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 234887ae2..3821d071f 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -113,7 +113,7 @@ message.timestamp.input.pattern= secor.compression.codec= # The secor file reader/writer used to read/write the data, by default we write sequence files -secor.file.reader.writer=com.pinterest.secor.io.impl.SequenceFileReaderWriter +secor.file.reader.writer.factory=com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory # 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 diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index ed584593f..360b6bfa2 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -16,7 +16,7 @@ */ package com.pinterest.secor.common; -import com.pinterest.secor.io.FileReaderWriter; +import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.ReflectionUtil; import com.pinterest.secor.util.StatsUtil; @@ -39,13 +39,13 @@ public class FileRegistry { private final SecorConfig mConfig; private HashMap> mFiles; - private HashMap mWriters; + private HashMap mWriters; private HashMap mCreationTimes; public FileRegistry(SecorConfig mConfig) { this.mConfig = mConfig; mFiles = new HashMap>(); - mWriters = new HashMap(); + mWriters = new HashMap(); mCreationTimes = new HashMap(); } @@ -82,9 +82,9 @@ public Collection getPaths(TopicPartition topicPartition) { * @return Writer for a given path. * @throws Exception */ - public FileReaderWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) + public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) throws Exception { - FileReaderWriter writer = mWriters.get(path); + FileWriter writer = mWriters.get(path); if (writer == null) { // Just in case. FileUtil.delete(path.getLogFilePath()); @@ -99,9 +99,7 @@ public FileReaderWriter getOrCreateWriter(LogFilePath path, CompressionCodec cod if (!files.contains(path)) { files.add(path); } - writer = ((FileReaderWriter) ReflectionUtil.createFileReaderWriter( - mConfig.getFileReaderWriter(), path, codec, - FileReaderWriter.Type.Writer)); + writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), path, codec); mWriters.put(path, writer); mCreationTimes.put(path, System.currentTimeMillis() / 1000L); LOG.debug("created writer for path " + path.getLogFilePath()); @@ -152,7 +150,7 @@ public void deleteTopicPartition(TopicPartition topicPartition) throws IOExcepti * @param path The path to remove the writer for. */ public void deleteWriter(LogFilePath path) throws IOException { - FileReaderWriter writer = mWriters.get(path); + FileWriter writer = mWriters.get(path); if (writer == null) { LOG.warn("No writer found for path " + path.getLogFilePath()); } else { @@ -190,7 +188,7 @@ public long getSize(TopicPartition topicPartition) throws IOException { Collection paths = getPaths(topicPartition); long result = 0; for (LogFilePath path : paths) { - FileReaderWriter writer = mWriters.get(path); + FileWriter writer = mWriters.get(path); if (writer != null) { result += writer.getLength(); } diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 3d5e6c1a8..33e6a7a57 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -215,8 +215,8 @@ public int getMaxMessageSizeBytes() { return getInt("secor.max.message.size.bytes"); } - public String getFileReaderWriter() { - return getString("secor.file.reader.writer"); + public String getFileReaderWriterFactory() { + return getString("secor.file.reader.writer.factory"); } public String getPerfTestTopicPrefix() { diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 73f8ed409..6905c17d3 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -67,8 +67,7 @@ private void init() throws Exception { mMessageReader = new MessageReader(mConfig, mOffsetTracker); FileRegistry fileRegistry = new FileRegistry(mConfig); mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, fileRegistry); - mMessageParser = (MessageParser) ReflectionUtil.createMessageParser( - mConfig.getMessageParserClass(), mConfig); + mMessageParser = ReflectionUtil.createMessageParser(mConfig.getMessageParserClass(), mConfig); mUploader = new Uploader(mConfig, mOffsetTracker, fileRegistry); mUnparsableMessages = 0.; } 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/FileReaderWriter.java b/src/main/java/com/pinterest/secor/io/FileWriter.java similarity index 53% rename from src/main/java/com/pinterest/secor/io/FileReaderWriter.java rename to src/main/java/com/pinterest/secor/io/FileWriter.java index 292af274d..07d2a819f 100644 --- a/src/main/java/com/pinterest/secor/io/FileReaderWriter.java +++ b/src/main/java/com/pinterest/secor/io/FileWriter.java @@ -1,12 +1,12 @@ /** * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * 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, @@ -18,56 +18,35 @@ import java.io.IOException; - /** - * Generic file reader/writer interface for all secor files - * - * All implementations should define the constructor with the signature: - * FileReaderWriter(LogFilePath path, CompressionCodec codec, FileReaderWriter.Type type) + * Generic file writer interface for for a particular type of Secor output file * - * example: public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec, - * FileReaderWriter.Type type) + * 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 FileReaderWriter { - - public enum Type { - Reader, - Writer; - } - - /** - * 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; - +public interface FileWriter { /** * Get length of data written up to now to the underlying file - * + * * @return - * @throws IOException + * @throws java.io.IOException */ public long getLength() throws IOException; /** * Write the given key and value to the file - * - * @param key - * @param value - * @throws IOException + * + * @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/impl/DelimitedTextFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java deleted file mode 100644 index c1b564674..000000000 --- a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriter.java +++ /dev/null @@ -1,123 +0,0 @@ -/** - * 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.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.compress.CompressionCodec; - -import com.google.common.io.CountingOutputStream; -import com.pinterest.secor.common.LogFilePath; -import com.pinterest.secor.io.FileReaderWriter; -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 DelimitedTextFileReaderWriter implements FileReaderWriter { - - // delimiter used between messages - private static final byte DELIMITER = '\n'; - - private final CountingOutputStream mCountingStream; - private final BufferedOutputStream mWriter; - - private final BufferedInputStream mReader; - private long mOffset; - - // constructor - public DelimitedTextFileReaderWriter(LogFilePath path, - CompressionCodec codec, FileReaderWriter.Type type) - throws FileNotFoundException, IOException { - - Path fsPath = new Path(path.getLogFilePath()); - FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); - if (type == FileReaderWriter.Type.Reader) { - InputStream inputStream = fs.open(fsPath); - this.mReader = (codec == null) ? new BufferedInputStream(inputStream) - : new BufferedInputStream( - codec.createInputStream(inputStream)); - this.mOffset = path.getOffset(); - this.mCountingStream = null; - this.mWriter = null; - } else if (type == FileReaderWriter.Type.Writer) { - this.mCountingStream = new CountingOutputStream(fs.create(fsPath)); - this.mWriter = (codec == null) ? new BufferedOutputStream( - this.mCountingStream) : new BufferedOutputStream( - codec.createOutputStream(this.mCountingStream)); - this.mReader = null; - } else { - throw new IllegalArgumentException("Undefined File Type: " + type); - } - } - - @Override - public void close() throws IOException { - if (this.mWriter != null) { - this.mWriter.close(); - } - if (this.mReader != null) { - this.mReader.close(); - } - } - - @Override - public long getLength() throws IOException { - assert this.mCountingStream != null; - return this.mCountingStream.getCount(); - } - - @Override - public void write(KeyValue keyValue) throws IOException { - assert this.mWriter != null; - this.mWriter.write(keyValue.getValue()); - this.mWriter.write(DELIMITER); - } - - @Override - public KeyValue next() throws IOException { - assert this.mReader != null; - 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()); - } - -} \ No newline at end of file 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..f0d1738c3 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java @@ -0,0 +1,125 @@ +/** + * 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 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; + + 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)); + 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(); + } + } + + protected class DelimitedTextFileWriter implements FileWriter { + private final CountingOutputStream mCountingStream; + private final BufferedOutputStream mWriter; + + 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)); + } + + @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()); + this.mWriter.write(DELIMITER); + } + + @Override + public void close() throws IOException { + this.mWriter.close(); + } + } +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java deleted file mode 100644 index 68300fa76..000000000 --- a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriter.java +++ /dev/null @@ -1,109 +0,0 @@ -/** - * 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 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 com.pinterest.secor.common.LogFilePath; -import com.pinterest.secor.io.FileReaderWriter; -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 SequenceFileReaderWriter implements FileReaderWriter { - - private final SequenceFile.Writer mWriter; - private final SequenceFile.Reader mReader; - private final LongWritable mKey; - private final BytesWritable mValue; - - // constructor - public SequenceFileReaderWriter(LogFilePath path, CompressionCodec codec, - FileReaderWriter.Type type) throws Exception { - Configuration config = new Configuration(); - Path fsPath = new Path(path.getLogFilePath()); - FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); - - if (type == FileReaderWriter.Type.Reader) { - this.mReader = new SequenceFile.Reader(fs, fsPath, config); - this.mKey = (LongWritable) mReader.getKeyClass().newInstance(); - this.mValue = (BytesWritable) mReader.getValueClass().newInstance(); - this.mWriter = null; - } else if (type == FileReaderWriter.Type.Writer) { - 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.mReader = null; - this.mKey = null; - this.mValue = null; - } else { - throw new IllegalArgumentException("Undefined File Type: " + type); - } - - } - - @Override - public void close() throws IOException { - if (this.mWriter != null) { - this.mWriter.close(); - } - if (this.mReader != null) { - this.mReader.close(); - } - } - - @Override - public long getLength() throws IOException { - return this.mWriter.getLength(); - } - - @Override - public void write(KeyValue keyValue) throws IOException { - LongWritable writeableKey = new LongWritable(keyValue.getKey()); - BytesWritable writeableValue = new BytesWritable(keyValue.getValue()); - this.mWriter.append(writeableKey, writeableValue); - } - - @Override - public KeyValue next() throws IOException { - if (mReader.next(mKey, mValue)) { - return new KeyValue(mKey.get(), mValue.getBytes()); - } else { - return null; - } - } - -} \ No newline at end of file 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..e009d2268 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java @@ -0,0 +1,120 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package 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 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 { + @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; + + public SequenceFileWriter(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, + 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(); + } + + @Override + public long getLength() throws IOException { + return this.mWriter.getLength(); + } + + @Override + public void write(KeyValue keyValue) throws IOException { + this.mKey.set(keyValue.getKey()); + 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(); + } + } +} \ No newline at end of file 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 58e4e7b97..531f29cbc 100644 --- a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java +++ b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java @@ -19,7 +19,7 @@ import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.common.TopicPartition; -import com.pinterest.secor.io.FileReaderWriter; +import com.pinterest.secor.io.FileReader; import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.util.CompressionUtil; import com.pinterest.secor.util.FileUtil; @@ -107,7 +107,7 @@ private void filterOffsets(long fromOffset, long toOffset) { } private int getMessageCount(LogFilePath logFilePath) throws Exception { - FileReaderWriter reader = createFileReaderWriter(logFilePath); + FileReader reader = createFileReader(logFilePath); int result = 0; while (reader.next() != null) { result++; @@ -156,7 +156,7 @@ public void verifyCounts(long fromOffset, long toOffset, int numMessages) throws } private void getOffsets(LogFilePath logFilePath, Set offsets) throws Exception { - FileReaderWriter reader = createFileReaderWriter(logFilePath); + FileReader reader = createFileReader(logFilePath); KeyValue record; while ((record = reader.next()) != null) { if (!offsets.add(record.getKey())) { @@ -202,16 +202,16 @@ public void verifySequences(long fromOffset, long toOffset) throws Exception { * @return * @throws Exception */ - private FileReaderWriter createFileReaderWriter(LogFilePath logFilePath) throws Exception { + private FileReader createFileReader(LogFilePath logFilePath) throws Exception { CompressionCodec codec = null; if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { codec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); } - FileReaderWriter fileReader = (FileReaderWriter) ReflectionUtil.createFileReaderWriter( - mConfig.getFileReaderWriter(), + FileReader fileReader = ReflectionUtil.createFileReader( + mConfig.getFileReaderWriterFactory(), logFilePath, - codec, - FileReaderWriter.Type.Reader); + codec + ); return fileReader; } } \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index 76ea305ff..6f5650deb 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -17,7 +17,8 @@ package com.pinterest.secor.uploader; import com.pinterest.secor.common.*; -import com.pinterest.secor.io.FileReaderWriter; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.util.CompressionUtil; import com.pinterest.secor.util.FileUtil; @@ -122,20 +123,20 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { * This method is intended to be overwritten in tests. * @throws Exception */ - protected FileReaderWriter createReader(LogFilePath srcPath, CompressionCodec codec) throws Exception { - return (FileReaderWriter) ReflectionUtil.createFileReaderWriter( - mConfig.getFileReaderWriter(), + protected FileReader createReader(LogFilePath srcPath, CompressionCodec codec) throws Exception { + return ReflectionUtil.createFileReader( + mConfig.getFileReaderWriterFactory(), srcPath, - codec, - FileReaderWriter.Type.Reader); + codec + ); } private void trim(LogFilePath srcPath, long startOffset) throws Exception { if (startOffset == srcPath.getOffset()) { return; } - FileReaderWriter reader = null; - FileReaderWriter 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. diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 10aa678ea..30420f8fe 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -18,10 +18,11 @@ import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.io.FileReaderWriter; - -import java.lang.reflect.Constructor; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.parser.MessageParser; import org.apache.hadoop.io.compress.CompressionCodec; /** @@ -29,40 +30,83 @@ * specified by name. * * @author Pawel Garbacki (pawel@pinterest.com) + * @author Silas Davis (github-code@silasdavis.net) */ public class ReflectionUtil { - public static Object createMessageParser(String className, - SecorConfig config) throws Exception { + /** + * 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())); + } - // Search for an "appropriate" constructor. - for (Constructor ctor : clazz.getConstructors()) { - Class[] paramTypes = ctor.getParameterTypes(); + // Assume that subclass of MessageParser has a constructor with the same signature as MessageParser + return (MessageParser) clazz.getConstructor(SecorConfig.class).newInstance(config); + } - // If the arity matches, let's use it. - if (paramTypes.length == 1) { - Object[] args = { config }; - return ctor.newInstance(args); - } + /** + * 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 + * @return a FileReaderWriterFactory with the runtime type of the class passed by name + * @throws Exception + */ + private static FileReaderWriterFactory createFileReaderWriterFactory(String className) throws Exception { + Class clazz = Class.forName(className); + if (!FileReaderWriterFactory.class.isAssignableFrom(clazz)) { + throw new IllegalArgumentException(String.format("The class '%s' is not assignable to '%s'.", + className, FileReaderWriterFactory.class.getName())); } - throw new IllegalArgumentException("Class not found " + className); + + // We assume a parameterless constructor + return (FileReaderWriterFactory) clazz.newInstance(); } - public static Object createFileReaderWriter(String className, - LogFilePath logFilePath, CompressionCodec compressionCodec, - FileReaderWriter.Type type) throws Exception { - Class clazz = Class.forName(className); - // Search for an "appropriate" constructor. - for (Constructor ctor : clazz.getConstructors()) { - Class[] paramTypes = ctor.getParameterTypes(); + /** + * 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 + * @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) + throws Exception { + return createFileReaderWriterFactory(className).BuildFileWriter(logFilePath, codec); + } - // If the arity matches, let's use it. - if (paramTypes.length == 3) { - Object[] args = { logFilePath, compressionCodec, type }; - return ctor.newInstance(args); - } - } - throw new IllegalArgumentException("Class not found " + className); + /** + * 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 + * @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) + throws Exception { + return createFileReaderWriterFactory(className).BuildFileReader(logFilePath, codec); } } \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index 80c81cde6..c56c5c7b1 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -17,7 +17,7 @@ package com.pinterest.secor.writer; import com.pinterest.secor.common.*; -import com.pinterest.secor.io.FileReaderWriter; +import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; @@ -82,7 +82,7 @@ public void write(ParsedMessage message) throws Exception { long offset = mOffsetTracker.getAdjustedCommittedOffsetCount(topicPartition); LogFilePath path = new LogFilePath(mLocalPrefix, mConfig.getGeneration(), offset, message, mFileExtension); - FileReaderWriter writer = mFileRegistry.getOrCreateWriter(path, mCodec); + FileWriter writer = mFileRegistry.getOrCreateWriter(path, mCodec); writer.write(new KeyValue(message.getOffset(), message.getPayload())); LOG.debug("appended message " + message + " to file " + path.getLogFilePath() + ". File length " + writer.getLength()); diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 35fa269c3..ccff161f5 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -36,7 +36,7 @@ PARENT_DIR=/tmp/secor_dev LOGS_DIR=${PARENT_DIR}/logs -S3_LOGS_DIR=s3://pinterest-dev/secor_dev +S3_LOGS_DIR=s3://sk-cloud-staging/secor_dev MESSAGES=1000 MESSAGE_TYPE=binary # For the compression tests to work, set this to the path of the Hadoop native libs. @@ -46,8 +46,8 @@ ADDITIONAL_OPTS= # various reader writer options to be used for testing declare -A READER_WRITERS -READER_WRITERS[json]=com.pinterest.secor.io.impl.DelimitedTextFileReaderWriter -READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriter +READER_WRITERS[json]=com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory +READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory # 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. @@ -253,7 +253,7 @@ post_and_verify_compressed_test() { for key in ${!READER_WRITERS[@]}; do MESSAGE_TYPE=${key} - ADDITIONAL_OPTS=-Dsecor.file.reader.writer=${READER_WRITERS[${key}]} + ADDITIONAL_OPTS=-Dsecor.file.reader.writer.factory=${READER_WRITERS[${key}]} echo "Running tests for Message Type: ${MESSAGE_TYPE} and ReaderWriter: ${READER_WRITERS[${key}]}" post_and_verify_test start_from_non_zero_offset_test diff --git a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java index 8aed5d0ca..021437cca 100644 --- a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java +++ b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java @@ -16,7 +16,7 @@ */ package com.pinterest.secor.common; -import com.pinterest.secor.io.FileReaderWriter; +import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.ReflectionUtil; @@ -55,8 +55,8 @@ public class FileRegistryTest extends TestCase { public void setUp() throws Exception { super.setUp(); PropertiesConfiguration properties = new PropertiesConfiguration(); - properties.addProperty("secor.file.reader.writer", - "com.pinterest.secor.io.impl.SequenceFileWriter"); + properties.addProperty("secor.file.reader.writer.factory", + "com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory"); SecorConfig secorConfig = new SecorConfig(properties); mRegistry = new FileRegistry(secorConfig); mLogFilePath = new LogFilePath("/some_parent_dir", PATH); @@ -68,18 +68,18 @@ private void createWriter() throws Exception { PowerMockito.mockStatic(FileUtil.class); PowerMockito.mockStatic(ReflectionUtil.class); - FileReaderWriter writer = Mockito.mock(FileReaderWriter.class); + FileWriter writer = Mockito.mock(FileWriter.class); Mockito.when( - ReflectionUtil.createFileReaderWriter( + ReflectionUtil.createFileWriter( Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class), - Mockito.any(FileReaderWriter.Type.class))) + Mockito.any(CompressionCodec.class) + )) .thenReturn(writer); Mockito.when(writer.getLength()).thenReturn(123L); - FileReaderWriter createdWriter = mRegistry.getOrCreateWriter( + FileWriter createdWriter = mRegistry.getOrCreateWriter( mLogFilePath, null); assertTrue(createdWriter == writer); } @@ -92,10 +92,10 @@ public void testGetOrCreateWriter() throws Exception { // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); - ReflectionUtil.createFileReaderWriter(Mockito.any(String.class), + ReflectionUtil.createFileWriter(Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class), - Mockito.any(FileReaderWriter.Type.class)); + Mockito.any(CompressionCodec.class) + ); PowerMockito.verifyStatic(); FileUtil.delete(PATH); @@ -118,18 +118,18 @@ private void createCompressedWriter() throws Exception { PowerMockito.mockStatic(FileUtil.class); PowerMockito.mockStatic(ReflectionUtil.class); - FileReaderWriter writer = Mockito.mock(FileReaderWriter.class); + FileWriter writer = Mockito.mock(FileWriter.class); Mockito.when( - ReflectionUtil.createFileReaderWriter( + ReflectionUtil.createFileWriter( Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class), - Mockito.any(FileReaderWriter.Type.class))) + Mockito.any(CompressionCodec.class) + )) .thenReturn(writer); Mockito.when(writer.getLength()).thenReturn(123L); - FileReaderWriter createdWriter = mRegistry.getOrCreateWriter( + FileWriter createdWriter = mRegistry.getOrCreateWriter( mLogFilePathGz, new GzipCodec()); assertTrue(createdWriter == writer); } @@ -146,10 +146,10 @@ public void testGetOrCreateWriterCompressed() throws Exception { FileUtil.delete(CRC_PATH); PowerMockito.verifyStatic(); - ReflectionUtil.createFileReaderWriter(Mockito.any(String.class), + ReflectionUtil.createFileWriter(Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class), - Mockito.any(FileReaderWriter.Type.class)); + Mockito.any(CompressionCodec.class) + ); TopicPartition topicPartition = new TopicPartition("some_topic", 0); Collection topicPartitions = mRegistry diff --git a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java b/src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java similarity index 70% rename from src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java rename to src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java index 3e378331d..0ef5b5742 100644 --- a/src/test/java/com/pinterest/secor/io/FileReaderWriterTest.java +++ b/src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java @@ -16,10 +16,7 @@ */ package com.pinterest.secor.io; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.InputStream; -import java.io.OutputStream; +import java.io.*; import java.net.URI; import org.apache.commons.configuration.PropertiesConfiguration; @@ -42,28 +39,27 @@ import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.io.impl.DelimitedTextFileReaderWriter; -import com.pinterest.secor.io.impl.SequenceFileReaderWriter; +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) * + * @author Praveen Murugesan (praveen@uber.com) */ @RunWith(PowerMockRunner.class) -@PrepareForTest({ FileSystem.class, DelimitedTextFileReaderWriter.class, - SequenceFile.class, SequenceFileReaderWriter.class, GzipCodec.class, - FileInputStream.class, FileOutputStream.class }) -public class FileReaderWriterTest extends TestCase { +@PrepareForTest({FileSystem.class, DelimitedTextFileReaderWriterFactory.class, + SequenceFile.class, SequenceFileReaderWriterFactory.class, GzipCodec.class, + FileInputStream.class, FileOutputStream.class}) +public class FileReaderWriterFactoryTest 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 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; @@ -78,19 +74,19 @@ public void setUp() throws Exception { private void setupSequenceFileReaderConfig() { PropertiesConfiguration properties = new PropertiesConfiguration(); - properties.addProperty("secor.file.reader.writer", - "com.pinterest.secor.io.impl.SequenceFileReaderWriter"); + properties.addProperty("secor.file.reader.writer.factory", + "com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory"); mConfig = new SecorConfig(properties); } - private void setupDelimitedTextFileReaderWriterConfig() { + private void setupDelimitedTextFileWriterConfig() { PropertiesConfiguration properties = new PropertiesConfiguration(); - properties.addProperty("secor.file.reader.writer", - "com.pinterest.secor.io.impl.DelimitedTextFileReaderWriter"); + properties.addProperty("secor.file.reader.writer.factory", + "com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory"); mConfig = new SecorConfig(properties); } - private void mockDelimitedTextFileReaderWriter(boolean isCompressed) throws Exception { + private void mockDelimitedTextFileWriter(boolean isCompressed) throws Exception { PowerMockito.mockStatic(FileSystem.class); FileSystem fs = Mockito.mock(FileSystem.class); Mockito.when( @@ -122,7 +118,7 @@ private void mockDelimitedTextFileReaderWriter(boolean isCompressed) throws Exce .thenReturn(outputStream); } - private void mockSequenceFileReaderWriter(boolean isCompressed) + private void mockSequenceFileWriter(boolean isCompressed) throws Exception { PowerMockito.mockStatic(FileSystem.class); FileSystem fs = Mockito.mock(FileSystem.class); @@ -141,9 +137,9 @@ private void mockSequenceFileReaderWriter(boolean isCompressed) .withArguments(Mockito.eq(fs), Mockito.eq(fsPath), Mockito.any(Configuration.class)).thenReturn(reader); - Mockito.> when(reader.getKeyClass()).thenReturn( + Mockito.>when(reader.getKeyClass()).thenReturn( (Class) LongWritable.class); - Mockito.> when(reader.getValueClass()).thenReturn( + Mockito.>when(reader.getValueClass()).thenReturn( (Class) BytesWritable.class); if (!isCompressed) { @@ -176,17 +172,15 @@ Mockito.> when(reader.getValueClass()).thenReturn( public void testSequenceFileReader() throws Exception { setupSequenceFileReaderConfig(); - mockSequenceFileReaderWriter(false); - ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePath, null, FileReaderWriter.Type.Reader); + mockSequenceFileWriter(false); + ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePath, null); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); FileSystem.get(Mockito.any(URI.class), Mockito.any(Configuration.class)); - mockSequenceFileReaderWriter(true); - ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Reader); + mockSequenceFileWriter(true); + ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), mLogFilePathGz, new GzipCodec()); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); @@ -196,11 +190,10 @@ public void testSequenceFileReader() throws Exception { public void testSequenceFileWriter() throws Exception { setupSequenceFileReaderConfig(); - mockSequenceFileReaderWriter(false); + mockSequenceFileWriter(false); - FileReaderWriter writer = (FileReaderWriter) ReflectionUtil - .createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePath, null, FileReaderWriter.Type.Writer); + FileWriter writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), + mLogFilePath, null); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); @@ -209,10 +202,10 @@ public void testSequenceFileWriter() throws Exception { assert writer.getLength() == 123L; - mockSequenceFileReaderWriter(true); + mockSequenceFileWriter(true); - writer = (FileReaderWriter) ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Writer); + writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), + mLogFilePathGz, new GzipCodec()); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); @@ -222,32 +215,32 @@ public void testSequenceFileWriter() throws Exception { assert writer.getLength() == 12L; } + public void testDelimitedTextFileWriter() throws Exception { - setupDelimitedTextFileReaderWriterConfig(); - mockDelimitedTextFileReaderWriter(false); - FileReaderWriter writer = (FileReaderWriter) ReflectionUtil - .createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePath, null, - FileReaderWriter.Type.Writer); + setupDelimitedTextFileWriterConfig(); + mockDelimitedTextFileWriter(false); + FileWriter writer = (FileWriter) ReflectionUtil + .createFileWriter(mConfig.getFileReaderWriterFactory(), + mLogFilePath, null + ); assert writer.getLength() == 0L; - mockDelimitedTextFileReaderWriter(true); - writer = (FileReaderWriter) ReflectionUtil - .createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePathGz, new GzipCodec(), - FileReaderWriter.Type.Writer); + mockDelimitedTextFileWriter(true); + writer = (FileWriter) ReflectionUtil + .createFileWriter(mConfig.getFileReaderWriterFactory(), + mLogFilePathGz, new GzipCodec() + ); assert writer.getLength() == 0L; } public void testDelimitedTextFileReader() throws Exception { - setupDelimitedTextFileReaderWriterConfig(); + setupDelimitedTextFileWriterConfig(); + + mockDelimitedTextFileWriter(false); - mockDelimitedTextFileReaderWriter(false); - ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePath, null, FileReaderWriter.Type.Reader); + ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePath, null); - mockDelimitedTextFileReaderWriter(true); - ReflectionUtil.createFileReaderWriter(mConfig.getFileReaderWriter(), - mLogFilePathGz, new GzipCodec(), FileReaderWriter.Type.Reader); + mockDelimitedTextFileWriter(true); + ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePathGz, new GzipCodec()); } } \ 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..eea747a78 --- /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.getKey(), kvout.getKey()); + assertArrayEquals(kv1.getValue(), kvout.getValue()); + kvout = fileReader.next(); + assertEquals(kv2.getKey(), kvout.getKey()); + assertArrayEquals(kv2.getValue(), kvout.getValue()); + } + + +} \ No newline at end of file diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index 3eb2ef9e4..3e834595c 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -17,7 +17,8 @@ package com.pinterest.secor.uploader; import com.pinterest.secor.common.*; -import com.pinterest.secor.io.FileReaderWriter; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.IdUtil; @@ -45,21 +46,21 @@ @PrepareForTest({ FileUtil.class, IdUtil.class }) public class UploaderTest extends TestCase { private static class TestUploader extends Uploader { - private FileReaderWriter mReader; + private FileReader mReader; public TestUploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, ZookeeperConnector zookeeperConnector) { super(config, offsetTracker, fileRegistry, zookeeperConnector); - mReader = Mockito.mock(FileReaderWriter.class); + mReader = Mockito.mock(FileReader.class); } @Override - protected FileReaderWriter createReader(LogFilePath srcPath, + protected FileReader createReader(LogFilePath srcPath, CompressionCodec codec) throws IOException { return mReader; } - public FileReaderWriter getReader() { + public FileReader getReader() { return mReader; } } @@ -175,7 +176,7 @@ public void testTrimFiles() throws Exception { Mockito.when(mFileRegistry.getPaths(mTopicPartition)).thenReturn( logFilePaths); - FileReaderWriter reader = mUploader.getReader(); + FileReader reader = mUploader.getReader(); Mockito.when(reader.next()).thenAnswer(new Answer() { private int mCallCount = 0; @@ -194,7 +195,7 @@ public KeyValue answer(InvocationOnMock invocation) Mockito.when(IdUtil.getLocalMessageDir()) .thenReturn("some_message_dir"); - FileReaderWriter writer = Mockito.mock(FileReaderWriter.class); + 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/" 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..553b58ef9 --- /dev/null +++ b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java @@ -0,0 +1,67 @@ +/** + * 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.parser.MessageParser; +import org.apache.commons.configuration.PropertiesConfiguration; +import org.junit.Test; + +public class ReflectionUtilTest { + + private SecorConfig mSecorConfig; + private LogFilePath mLogFilePath; + + public void setUp() throws Exception { + PropertiesConfiguration properties = new PropertiesConfiguration(); + 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); + } + + @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); + } +} \ No newline at end of file From c3cb145915525d8083ac60e0cf5180c658190de2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mathias=20S=C3=B6derberg?= Date: Mon, 3 Nov 2014 10:04:21 +0100 Subject: [PATCH 048/330] Check upload policy every N messages / seconds MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reusing `secor.messages.per.second` which is used for rate limiting, but we’re currently setting it to a very large number to avoid being rate limited at all, so we might as well use it for this as well. --- src/main/java/com/pinterest/secor/consumer/Consumer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 6905c17d3..20c830b56 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -83,6 +83,7 @@ public void run() { } // 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) { @@ -92,7 +93,7 @@ public void run() { } long now = System.currentTimeMillis(); - if (nMessages++ % 10000 == 0 || + if (nMessages++ % checkMessagesPerSecond == 0 || (now - lastChecked) > checkEveryNSeconds * 1000) { lastChecked = now; checkUploadPolicy(); From d1a8b0ea813a30f43ee6c12fcb10beb92b606acb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mathias=20S=C3=B6derberg?= Date: Thu, 11 Dec 2014 16:42:51 +0100 Subject: [PATCH 049/330] Add support for additional Kafka fetch options --- src/main/config/secor.common.properties | 6 ++++++ src/main/java/com/pinterest/secor/common/SecorConfig.java | 8 ++++++++ .../java/com/pinterest/secor/reader/MessageReader.java | 6 ++++++ 3 files changed, 20 insertions(+) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 3821d071f..cafa86fcb 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -47,6 +47,12 @@ 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 33e6a7a57..2cb7c3b9d 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -103,6 +103,14 @@ public String getSocketReceieveBufferBytes() { 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 int getGeneration() { return getInt("secor.generation"); } diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index c24d9ef4b..12ccc22ff 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -125,6 +125,12 @@ private ConsumerConfig createConsumerConfig() throws UnknownHostException { 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); } From b1c620ed26c635e91b5d78b73f49ba1d997136f9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mathias=20So=CC=88derberg?= Date: Fri, 31 Oct 2014 21:57:30 +0100 Subject: [PATCH 050/330] (Fully) local integration tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit introduces a way to run the integration tests in a purely local fashion, using `fakes3` (a Ruby gem) rather than using an actual S3 bucket. In order to accomplish this a few configuration options needs to be set (AWS keys and S3 bucket), and I’ve included “defaults” that will work on Travis (in separate config. files). One also needs to add an entry to `/etc/hosts` (or equivalent), that points `test-bucket.localhost` to `127.0.0.1`. I’ve added a rather naive function, `check_for_native_libs`, that checks whether `HADOOP_NATIVE_LIB_PATH` actually contains any `*.so` files, and if it doesn’t it’ll ignore the “compressed tests” for sequence files, as it's not possible to run them without the native libs, and it's pretty darn difficult to build them on Mac OS X, not even sure if it's supported at all. Furthermore there’s a Makefile that simplifies running the integration tests, a simple `make integration` is all that’s needed. It’ll build, package and extract Secor to a directory in `/tmp` and the run the tests. Perhaps there should be a section in the README dedicated to how the test setup works, and how to run the tests. --- .travis.yml | 20 ++++- Makefile | 22 +++++ src/main/config/secor.test.backup.properties | 3 + .../config/secor.test.partition.properties | 3 + src/main/config/secor.test.properties | 4 + src/main/scripts/run_common.sh | 4 + src/main/scripts/run_kafka_class.sh | 16 ---- src/main/scripts/run_tests.sh | 89 ++++++++++++++----- src/main/scripts/run_zookeeper_command.sh | 2 +- src/test/config/jets3t.properties | 3 + src/test/config/test.s3cfg | 12 +++ 11 files changed, 135 insertions(+), 43 deletions(-) create mode 100644 Makefile create mode 100644 src/main/config/secor.test.backup.properties create mode 100644 src/main/config/secor.test.partition.properties create mode 100644 src/main/config/secor.test.properties create mode 100644 src/test/config/jets3t.properties create mode 100644 src/test/config/test.s3cfg diff --git a/.travis.yml b/.travis.yml index b41b756f0..263b872b6 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,6 +1,20 @@ language: java +addons: + hosts: + - test-bucket.localhost +env: + - PATH=$PATH:$HOME/.s3cmd SECOR_LOCAL_S3=true S3CMD=1.0.1 jdk: - - oraclejdk8 + - openjdk7 - oraclejdk7 - - openjdk6 - - openjdk7 \ No newline at end of file + - 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/Makefile b/Makefile new file mode 100644 index 000000000..b85de0ce7 --- /dev/null +++ b/Makefile @@ -0,0 +1,22 @@ +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 + +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)/jets3t.properties ] && jar uf $(TEST_HOME)/secor-*.jar -C $(TEST_CONFIG) jets3t.properties + cd $(TEST_HOME) && ./scripts/run_tests.sh + +test: build unit integration + 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.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/scripts/run_common.sh b/src/main/scripts/run_common.sh index 46715a060..73b21a697 100755 --- a/src/main/scripts/run_common.sh +++ b/src/main/scripts/run_common.sh @@ -12,3 +12,7 @@ if [ -z "${JAVA_HOME}" ]; then else JAVA="${JAVA_HOME}/bin/java" fi + +DEFAULT_CLASSPATH="*:lib/*" +CLASSPATH=${CLASSPATH:-$DEFAULT_CLASSPATH} + diff --git a/src/main/scripts/run_kafka_class.sh b/src/main/scripts/run_kafka_class.sh index bc41ec211..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 " diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index ccff161f5..6a021ee32 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -36,7 +36,8 @@ PARENT_DIR=/tmp/secor_dev LOGS_DIR=${PARENT_DIR}/logs -S3_LOGS_DIR=s3://sk-cloud-staging/secor_dev +BUCKET=${SECOR_BUCKET:-test-bucket} +S3_LOGS_DIR=s3://${BUCKET}/secor_dev MESSAGES=1000 MESSAGE_TYPE=binary # For the compression tests to work, set this to the path of the Hadoop native libs. @@ -51,68 +52,100 @@ READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriterFacto # 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) +WAIT_TIME=${SECOR_WAIT_TIME:-120} +BASE_DIR=$(dirname $0) +CONF_DIR=${BASE_DIR}/.. -source ${base_dir}/run_common.sh +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 [ -n ${SECOR_LOCAL_S3} ]; then + run_command "s3cmd -c ${CONF_DIR}/test.s3cfg ls ${S3_LOGS_DIR} | awk '{ print \$4 }' | xargs -L 1 s3cmd -c ${CONF_DIR}/test.s3cfg del" + else + run_command "s3cmd del --recursive ${S3_LOGS_DIR}" + 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 2 + 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 -9 'fakes3' > /dev/null 2>&1 || 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 ${ADDITIONAL_OPTS} -cp secor-0.1-SNAPSHOT.jar:lib/* \ + -Dconfig=secor.test.backup.properties ${ADDITIONAL_OPTS} -cp $CLASSPATH \ com.pinterest.secor.main.ConsumerMain > ${LOGS_DIR}/secor_backup.log 2>&1 &" if [ "${MESSAGE_TYPE}" = "binary" ]; then run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.partition.properties ${ADDITIONAL_OPTS} -cp secor-0.1-SNAPSHOT.jar:lib/* \ + -Dconfig=secor.test.partition.properties ${ADDITIONAL_OPTS} -cp $CLASSPATH \ com.pinterest.secor.main.ConsumerMain > ${LOGS_DIR}/secor_partition.log 2>&1 &" fi } stop_secor() { - run_command "pkill -f 'com.pinterest.secor.main.ConsumerMain' | true" + run_command "pkill -f 'com.pinterest.secor.main.ConsumerMain' || true" } 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() { run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + -Dconfig=secor.test.backup.properties -cp ${CLASSPATH} \ com.pinterest.secor.main.TestLogMessageProducerMain -t test -m $1 -p 1 -type ${MESSAGE_TYPE} > \ ${LOGS_DIR}/test_log_message_producer.log 2>&1" } @@ -124,7 +157,7 @@ verify() { fi for RUNMODE in ${RUNMODE_0} ${RUNMODE_1}; do run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.${RUNMODE}.properties ${ADDITIONAL_OPTS} -cp secor-0.1-SNAPSHOT.jar:lib/* \ + -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=$? @@ -132,6 +165,8 @@ verify() { 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 + stop_all + stop_s3 exit ${VERIFICATION_EXIT_CODE} fi done @@ -140,16 +175,15 @@ verify() { set_offsets_in_zookeeper() { for group in secor_backup secor_partition; do for partition in 0 1; do - run_command "${base_dir}/run_zookeeper_command.sh localhost:2181 create \ + run_command "${BASE_DIR}/run_zookeeper_command.sh localhost:2181 create \ /consumers \'\' > ${LOGS_DIR}/run_zookeeper_command.log 2>&1" - run_command "${base_dir}/run_zookeeper_command.sh localhost:2181 create \ + 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 \ + 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 \ + 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 \ + run_command "${BASE_DIR}/run_zookeeper_command.sh localhost:2181 create \ /consumers/${group}/offsets/test/${partition} $1 > \ ${LOGS_DIR}/run_zookeeper_command.log 2>&1" done @@ -250,6 +284,8 @@ post_and_verify_compressed_test() { echo -e "\e[1;42;97mpost_and_verify_compressed_test succeeded\e[0m" } +check_for_native_libs +start_s3 for key in ${!READER_WRITERS[@]}; do MESSAGE_TYPE=${key} @@ -258,6 +294,13 @@ for key in ${!READER_WRITERS[@]}; do post_and_verify_test start_from_non_zero_offset_test move_offset_back_test - post_and_verify_compressed_test + if [ ${key} = "json" ]; then + post_and_verify_compressed_test + elif [ -z ${SKIP_COMPRESSED_BINARY} ]; then + post_and_verify_compressed_test + else + echo "Skipping compressed tests for ${key}" + fi done +stop_s3 diff --git a/src/main/scripts/run_zookeeper_command.sh b/src/main/scripts/run_zookeeper_command.sh index 6dca38e93..178b92ada 100755 --- a/src/main/scripts/run_zookeeper_command.sh +++ b/src/main/scripts/run_zookeeper_command.sh @@ -25,4 +25,4 @@ 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/jets3t.properties b/src/test/config/jets3t.properties new file mode 100644 index 000000000..ecd21a143 --- /dev/null +++ b/src/test/config/jets3t.properties @@ -0,0 +1,3 @@ +s3service.https-only=false +s3service.s3-endpoint-http-port=5000 +s3service.s3-endpoint=localhost 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 From deb83deaad76d4c289bb05f3e9d7fc5d7d5be164 Mon Sep 17 00:00:00 2001 From: Mulloy Morrow Date: Wed, 11 Mar 2015 10:15:05 -0700 Subject: [PATCH 051/330] Update pom.xml up guava version dependency --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index fc4560e3d..069085186 100644 --- a/pom.xml +++ b/pom.xml @@ -132,7 +132,7 @@ com.google.guava guava - 14.0 + 18.0 net.minidev From eb6277eee2ac9000a82ed744b468ba854906d7ac Mon Sep 17 00:00:00 2001 From: Pawel Garbacki Date: Thu, 12 Mar 2015 20:48:45 -0700 Subject: [PATCH 052/330] Update README.md --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index 3a8bc86e1..c8082f488 100644 --- a/README.md +++ b/README.md @@ -1,5 +1,7 @@ # Pinterest Secor +[![Build Status](https://travis-ci.org/pinterest/secor.svg)](https://travis-ci.org/pinterest/secor) + Secor is a service persisting [Kafka] logs to [Amazon S3]. ## Key features From 54e22a3943812cc0dad33144d14a645e2aa3f38d Mon Sep 17 00:00:00 2001 From: Pawel Garbacki Date: Sun, 29 Mar 2015 20:44:59 -0700 Subject: [PATCH 053/330] Update secor.common.properties --- src/main/config/secor.common.properties | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index cafa86fcb..3250bef62 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -115,7 +115,9 @@ message.timestamp.name=timestamp # 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'. +# 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= # The secor file reader/writer used to read/write the data, by default we write sequence files From 4d097e9dbaa224967dcdbd38b0eae0f2b1d9c0ff Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Fri, 10 Apr 2015 13:08:53 -0400 Subject: [PATCH 054/330] fix bug in casting stat to Long --- src/main/java/com/pinterest/secor/common/FileRegistry.java | 2 +- src/main/java/com/pinterest/secor/tools/ProgressMonitor.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index 360b6bfa2..ac0c05476 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -43,7 +43,7 @@ public class FileRegistry { private HashMap mCreationTimes; public FileRegistry(SecorConfig mConfig) { - this.mConfig = mConfig; + this.mConfig = mConfig; mFiles = new HashMap>(); mWriters = new HashMap(); mCreationTimes = new HashMap(); diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 693433f7d..1e7c7850a 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -155,7 +155,7 @@ private void exportToStatsD(List stats) { .append(PERIOD) .append(tags.get(Stat.STAT_KEYS.PARTITION.getName())) .toString(); - client.recordGaugeValue(aspect, (Long)stat.get(Stat.STAT_KEYS.VALUE.getName())); + client.recordGaugeValue(aspect, Long.parseLong((String)stat.get(Stat.STAT_KEYS.VALUE.getName()))); } } From 4eeef1325f99cc2141f1608790762dfc6920da09 Mon Sep 17 00:00:00 2001 From: Terry Bates Date: Mon, 13 Apr 2015 02:08:18 -0700 Subject: [PATCH 055/330] Changed "compromized" to "compromised." This typo was located in the "strong consistency" bullet, under the "key features" heading. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index c8082f488..5033b9af2 100644 --- a/README.md +++ b/README.md @@ -5,7 +5,7 @@ Secor is a service persisting [Kafka] logs to [Amazon S3]. ## Key features - - **strong consistency**: as long as [Kafka] is not dropping messages (e.g., due to aggresive cleanup policy) before Secor is able to read them, it is guaranteed that each message will be saved in exacly one [S3] file. This property is not compromized by the notorious temporal inconsisteny of [S3] caused by the [eventual consistency] model, + - **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 compromised by the notorious temporal inconsisteny of [S3] caused by the [eventual consistency] model, - **fault tolerance**: any component of Secor is allowed to crash at any given point without compromising data integrity, - **load distribution**: Secor may be distributed across multiple machines, - **horizontal scalability**: scaling the system out to handle more load is as easy as starting extra Secor processes. Reducing the resource footprint can be achieved by killing any of the running Secor processes. Neither ramping up nor down has any impact on data consistency, From 98e504bec7d144a094f5a877657889db05618fef Mon Sep 17 00:00:00 2001 From: Dmitry Kislyuk Date: Sun, 20 Jul 2014 07:25:47 -0700 Subject: [PATCH 056/330] Update README.md Minor text changes --- README.md | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 5033b9af2..de6610b5c 100644 --- a/README.md +++ b/README.md @@ -5,13 +5,13 @@ Secor is a service persisting [Kafka] logs to [Amazon S3]. ## Key features - - **strong consistency**: as long as [Kafka] is not dropping messages (e.g., due to aggresive cleanup policy) before Secor is able to read them, it is guaranteed that each message will be saved in exacly one [S3] file. This property is not compromised by the notorious temporal inconsisteny of [S3] caused by the [eventual consistency] model, + - **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], - **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] / [statsD], + - **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, - **Qubole interface**: Secor connects to [Qubole] to add finalized output partitions to Hive tables. @@ -51,7 +51,7 @@ 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 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 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). You may override the field used to extract the timestamp by setting 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. @@ -70,7 +70,7 @@ Currently secor supports the following output formats - **Delimited Text Files**: A new line delimited raw text file. ## 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. @@ -87,7 +87,7 @@ 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 @@ -133,4 +133,3 @@ If you have any questions or comments, you can reach us at [secor-users@googlegr [OpenTSDB]: http://opentsdb.net/ [Qubole]: http://www.qubole.com/ [statsD]: https://github.com/etsy/statsd/ - From 215416265c900891f82558ae7e798ed3d1c4dd71 Mon Sep 17 00:00:00 2001 From: Jon Parise Date: Wed, 13 May 2015 06:11:43 -0700 Subject: [PATCH 057/330] ThriftMessageParser expects field id 1, not 0. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index de6610b5c..5d579a469 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ 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 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). 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. 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). You may override the field used to extract the timestamp by setting 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. From 1d3d322ff909c41380c7d827ac5d8826cf43a793 Mon Sep 17 00:00:00 2001 From: Ramki Venkatachalam Date: Fri, 15 May 2015 09:13:23 -0700 Subject: [PATCH 058/330] adding stats counter for rebalance --- .../java/com/pinterest/secor/util/StatsUtil.java | 5 +++++ .../com/pinterest/secor/writer/MessageWriter.java | 15 +++++++++------ 2 files changed, 14 insertions(+), 6 deletions(-) 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/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index c56c5c7b1..0b46058be 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -16,22 +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.common.SecorConfig; 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.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. * @@ -66,6 +68,7 @@ public void adjustOffset(Message message) throws IOException { 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 + From b24f4eafc635d7976106a87cc8fc79907ea75457 Mon Sep 17 00:00:00 2001 From: Yejun Yang Date: Tue, 19 May 2015 17:43:00 -0700 Subject: [PATCH 059/330] avoid message toString when debug not enabled --- src/main/java/com/pinterest/secor/reader/MessageReader.java | 2 +- src/main/java/com/pinterest/secor/writer/MessageWriter.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 12ccc22ff..a88ecbf6c 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -150,7 +150,7 @@ public Message read() { updateAccessTime(topicPartition); // Skip already committed messages. long committedOffsetCount = mOffsetTracker.getTrueCommittedOffsetCount(topicPartition); - LOG.debug("read message" + message); + LOG.debug("read message {}", message); exportStats(); if (message.getOffset() < committedOffsetCount) { LOG.debug("skipping message message " + message + " because its offset precedes " + diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index c56c5c7b1..dd4c3f8a3 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -84,7 +84,7 @@ public void write(ParsedMessage message) throws Exception { mFileExtension); FileWriter writer = mFileRegistry.getOrCreateWriter(path, mCodec); writer.write(new KeyValue(message.getOffset(), message.getPayload())); - LOG.debug("appended message " + message + " to file " + path.getLogFilePath() + - ". File length " + writer.getLength()); + LOG.debug("appended message {} to file {}. File length {}", + message, path.getLogFilePath(), writer.getLength()); } } From 7b951a465ec328fd792b6c95b891ab14041eb715 Mon Sep 17 00:00:00 2001 From: Yejun Yang Date: Sat, 23 May 2015 20:15:25 +0000 Subject: [PATCH 060/330] fix delimited file memory leak --- .../DelimitedTextFileReaderWriterFactory.java | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java index f0d1738c3..879c5d267 100644 --- a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java @@ -29,6 +29,9 @@ 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; @@ -57,6 +60,7 @@ public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec code protected class DelimitedTextFileReader implements FileReader { private final BufferedInputStream mReader; private long mOffset; + private Decompressor mDecompressor; public DelimitedTextFileReader(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -64,7 +68,8 @@ public DelimitedTextFileReader(LogFilePath path, CompressionCodec codec) throws InputStream inputStream = fs.open(fsPath); this.mReader = (codec == null) ? new BufferedInputStream(inputStream) : new BufferedInputStream( - codec.createInputStream(inputStream)); + codec.createInputStream(inputStream, + mDecompressor = CodecPool.getDecompressor(codec))); this.mOffset = path.getOffset(); } @@ -89,12 +94,16 @@ public KeyValue next() throws IOException { @Override public void close() throws IOException { this.mReader.close(); + if (mDecompressor != null) { + CodecPool.returnDecompressor(mDecompressor); + } } } protected class DelimitedTextFileWriter implements FileWriter { private final CountingOutputStream mCountingStream; private final BufferedOutputStream mWriter; + private Compressor mCompressor; public DelimitedTextFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -102,7 +111,8 @@ public DelimitedTextFileWriter(LogFilePath path, CompressionCodec codec) throws this.mCountingStream = new CountingOutputStream(fs.create(fsPath)); this.mWriter = (codec == null) ? new BufferedOutputStream( this.mCountingStream) : new BufferedOutputStream( - codec.createOutputStream(this.mCountingStream)); + codec.createOutputStream(this.mCountingStream, + mCompressor = CodecPool.getCompressor(codec))); } @Override @@ -120,6 +130,9 @@ public void write(KeyValue keyValue) throws IOException { @Override public void close() throws IOException { this.mWriter.close(); + if (mCompressor != null) { + CodecPool.returnCompressor(mCompressor); + } } } -} \ No newline at end of file +} From 18f4af26f6e4424c4f969fd44a81cf3b9ca2701f Mon Sep 17 00:00:00 2001 From: Yejun Yang Date: Sat, 23 May 2015 20:26:29 +0000 Subject: [PATCH 061/330] more close to SequenceFile implmentation --- .../impl/DelimitedTextFileReaderWriterFactory.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java index 879c5d267..42e7e9d20 100644 --- a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java @@ -60,7 +60,7 @@ public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec code protected class DelimitedTextFileReader implements FileReader { private final BufferedInputStream mReader; private long mOffset; - private Decompressor mDecompressor; + private Decompressor mDecompressor = null; public DelimitedTextFileReader(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -94,16 +94,15 @@ public KeyValue next() throws IOException { @Override public void close() throws IOException { this.mReader.close(); - if (mDecompressor != null) { - CodecPool.returnDecompressor(mDecompressor); - } + CodecPool.returnDecompressor(mDecompressor); + mDecompressor = null; } } protected class DelimitedTextFileWriter implements FileWriter { private final CountingOutputStream mCountingStream; private final BufferedOutputStream mWriter; - private Compressor mCompressor; + private Compressor mCompressor = null; public DelimitedTextFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -130,9 +129,8 @@ public void write(KeyValue keyValue) throws IOException { @Override public void close() throws IOException { this.mWriter.close(); - if (mCompressor != null) { - CodecPool.returnCompressor(mCompressor); - } + CodecPool.returnCompressor(mCompressor); + mCompressor = null; } } } From 1aaaaf6c536f550e5c1eb1ef6d9a85552a7f1f0d Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Wed, 1 Jul 2015 11:37:13 -0500 Subject: [PATCH 062/330] Support a prefix for secor data in zookeeper. --- src/main/config/secor.common.properties | 3 +++ .../com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../com/pinterest/secor/uploader/Uploader.java | 14 ++++++++++++-- .../com/pinterest/secor/uploader/UploaderTest.java | 1 + 4 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 3250bef62..f6c9c2f9b 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -32,6 +32,9 @@ aws.secret.key= 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 2cb7c3b9d..2e03856f2 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -231,6 +231,10 @@ public String getPerfTestTopicPrefix() { return getString("secor.kafka.perf_topic_prefix"); } + public String getZookeeperPath() { + return getString("secor.zookeeper.path"); + } + private void checkProperty(String name) { if (!mProperties.containsKey(name)) { throw new RuntimeException("Failed to find required configuration option '" + diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index 6f5650deb..fdfa58711 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -16,6 +16,7 @@ */ package com.pinterest.secor.uploader; +import com.google.common.base.Joiner; import com.pinterest.secor.common.*; import com.pinterest.secor.io.FileReader; import com.pinterest.secor.io.FileWriter; @@ -25,6 +26,7 @@ import com.pinterest.secor.util.IdUtil; import com.pinterest.secor.util.ReflectionUtil; +import org.apache.commons.lang.StringUtils; import org.apache.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,8 +92,16 @@ public void run() { private void uploadFiles(TopicPartition topicPartition) throws Exception { long committedOffsetCount = mOffsetTracker.getTrueCommittedOffsetCount(topicPartition); long lastSeenOffset = mOffsetTracker.getLastSeenOffset(topicPartition); - final String lockPath = "/secor/locks/" + topicPartition.getTopic() + "/" + - topicPartition.getPartition(); + + String stripped = StringUtils.strip(mConfig.getZookeeperPath(), "/"); + final String lockPath = Joiner.on("/").skipNulls().join( + "", + stripped.isEmpty() ? null : stripped, + "secor", + "locks", + topicPartition.getTopic(), + topicPartition.getPartition()); + // Deleting writers closes their streams flushing all pending data to the disk. mFileRegistry.deleteWriters(topicPartition); mZookeeperConnector.lock(lockPath); diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index 3e834595c..a6a90e1a1 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -88,6 +88,7 @@ public void setUp() throws Exception { 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); From 942c3f70a334b728baaade5e86addd8f8286e6d0 Mon Sep 17 00:00:00 2001 From: Christian Nguyen Van Than Date: Thu, 9 Jul 2015 11:39:35 +0200 Subject: [PATCH 063/330] avoid string concatenation in slf4j logs Conflicts: src/main/java/com/pinterest/secor/parser/DateMessageParser.java --- .../pinterest/secor/common/FileRegistry.java | 11 ++++---- .../pinterest/secor/common/KafkaClient.java | 10 +++----- .../pinterest/secor/common/OffsetTracker.java | 9 +++---- .../secor/common/ZookeeperConnector.java | 12 ++++----- .../pinterest/secor/consumer/Consumer.java | 2 +- .../pinterest/secor/main/ConsumerMain.java | 4 +-- .../secor/parser/DateMessageParser.java | 6 ++--- .../secor/parser/PartitionFinalizer.java | 21 ++++++---------- .../pinterest/secor/reader/MessageReader.java | 4 +-- .../pinterest/secor/tools/LogFileDeleter.java | 6 ++--- .../secor/tools/ProgressMonitor.java | 16 ++++++------ .../pinterest/secor/uploader/Uploader.java | 25 ++++++++----------- .../pinterest/secor/writer/MessageWriter.java | 8 +++--- 13 files changed, 58 insertions(+), 76 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index ac0c05476..98b2520a2 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -102,7 +102,7 @@ public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), path, codec); 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()); } return writer; } @@ -152,9 +152,9 @@ public void deleteTopicPartition(TopicPartition topicPartition) throws IOExcepti public void deleteWriter(LogFilePath path) throws IOException { 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); @@ -168,8 +168,7 @@ public void deleteWriter(LogFilePath path) throws IOException { public void deleteWriters(TopicPartition topicPartition) throws IOException { HashSet paths = mFiles.get(topicPartition); if (paths == null) { - LOG.warn("No paths found for topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition()); + LOG.warn("No paths found for topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); } else { for (LogFilePath path : paths) { deleteWriter(path); @@ -212,7 +211,7 @@ public long getModificationAgeSec(TopicPartition topicPartition) throws IOExcept 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; diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index b7edc61b1..ab33ef149 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -59,8 +59,7 @@ public KafkaClient(SecorConfig config) { private HostAndPort findLeader(TopicPartition topicPartition) { SimpleConsumer consumer = null; try { - LOG.info("looking up leader for topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition()); + LOG.info("looking up leader for topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); consumer = new SimpleConsumer(mConfig.getKafkaSeedBrokerHost(), mConfig.getKafkaSeedBrokerPort(), 100000, 64 * 1024, "leaderLookup"); @@ -113,8 +112,8 @@ 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); + LOG.info("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) @@ -138,8 +137,7 @@ private Message getMessage(TopicPartition topicPartition, long offset, public SimpleConsumer createConsumer(TopicPartition topicPartition) { HostAndPort leader = findLeader(topicPartition); - LOG.info("leader for topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition() + " is " + leader.toString()); + LOG.info("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); diff --git a/src/main/java/com/pinterest/secor/common/OffsetTracker.java b/src/main/java/com/pinterest/secor/common/OffsetTracker.java index 7aa976cf8..8fff6eb70 100644 --- a/src/main/java/com/pinterest/secor/common/OffsetTracker.java +++ b/src/main/java/com/pinterest/secor/common/OffsetTracker.java @@ -52,12 +52,11 @@ public long setLastSeenOffset(TopicPartition topicPartition, long offset) { mLastSeenOffset.put(topicPartition, offset); if (lastSeenOffset + 1 != offset) { if (lastSeenOffset >= 0) { - LOG.warn("offset for topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition() + " changed from " + lastSeenOffset + " to " + - offset); + LOG.warn("offset for topic {} partition {} changed from {} to {}", + topicPartition.getTopic(),topicPartition.getPartition(),lastSeenOffset, offset); } else { - LOG.info("starting to consume topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition() + " from offset " + offset); + LOG.info("starting to consume topic {} partition from offset {}", + topicPartition.getTopic(),topicPartition.getPartition(),offset); } } if (mFirstSeendOffset.get(topicPartition) == null) { diff --git a/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java b/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java index 62287c94c..e86a31200 100644 --- a/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java +++ b/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java @@ -116,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; } } @@ -156,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) { } } @@ -166,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) { @@ -184,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); } } @@ -193,7 +193,7 @@ 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); } diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 20c830b56..969819488 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -142,7 +142,7 @@ private boolean consumeNextMessage() { if (mUnparsableMessages > MAX_UNPARSABLE_MESSAGES) { throw new RuntimeException("Failed to parse message " + rawMessage, e); } - LOG.warn("Failed to parse message " + rawMessage, e); + LOG.warn("Failed to parse message {}", rawMessage, e); } if (parsedMessage != null) { 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/parser/DateMessageParser.java b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java index 3370b9352..be602046c 100644 --- a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java @@ -62,9 +62,9 @@ public String[] extractPartitions(Message message) { result[0] = "dt=" + outputFormatter.format(dateFormat); return result; } 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 = {} for the input pattern = {} . Using date default = {}", + fieldValue.toString(), inputPattern.toString(), result[0]); } } } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 558d0fd41..b826db3ae 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -67,8 +67,7 @@ private long getLastTimestampMillis(TopicPartition topicPartition) throws Except Message message = mKafkaClient.getLastMessage(topicPartition); if (message == null) { // This will happen if no messages have been posted to the given topic partition. - LOG.error("No message found for topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition()); + LOG.error("No message found for topic {} partition {}" + topicPartition.getTopic(), topicPartition.getPartition()); return -1; } return mMessageParser.extractTimestampMillis(message); @@ -93,8 +92,7 @@ private long getLastTimestampMillis(String topic) throws Exception { 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()); + LOG.error("No message found for topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); return -1; } return mMessageParser.extractTimestampMillis(message); @@ -164,11 +162,10 @@ private void finalizePartitionsUpTo(String topic, Calendar calendar) throws IOEx try { mQuboleClient.addPartition(mConfig.getHivePrefix() + topic, "dt='" + partitionStr + "'"); } catch (Exception e) { - LOG.error("failed to finalize topic " + topic + " partition dt=" + partitionStr, - e); + LOG.error("failed to finalize topic {} partition dt={}", topic , partitionStr, e); continue; } - LOG.info("touching file " + successFilePath); + LOG.info("touching file {}", successFilePath); FileUtil.touch(successFilePath); } } @@ -199,8 +196,7 @@ private long getFinalizedTimestampMillis(String topic) throws Exception { 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); + LOG.info("finalized timestamp for topic {} partition {} is {}", topic, partition, timestamp); if (timestamp == -1) { return -1; } else { @@ -219,12 +215,11 @@ 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); + LOG.info("finalizing topic {}", topic); long finalizedTimestampMillis = getFinalizedTimestampMillis(topic); - LOG.info("finalized timestamp for topic " + topic + " is " + - finalizedTimestampMillis); + LOG.info("finalized timestamp for topic {} is {}", topic , finalizedTimestampMillis); if (finalizedTimestampMillis != -1) { Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); calendar.setTimeInMillis(finalizedTimestampMillis); diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index a88ecbf6c..245320346 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -153,8 +153,8 @@ public Message read() { LOG.debug("read message {}", message); 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/tools/LogFileDeleter.java b/src/main/java/com/pinterest/secor/tools/LogFileDeleter.java index 68535da85..799ae69f2 100644 --- a/src/main/java/com/pinterest/secor/tools/LogFileDeleter.java +++ b/src/main/java/com/pinterest/secor/tools/LogFileDeleter.java @@ -48,13 +48,11 @@ public void deleteOldLogs() throws Exception { 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/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 1e7c7850a..032ad482a 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -115,7 +115,7 @@ private void makeRequest(String body) throws IOException { private void exportToTsdb(Stat stat) throws IOException { - LOG.info("exporting metric to tsdb " + stat); + LOG.info("exporting metric to tsdb {}", stat); makeRequest(stat.toString()); } @@ -166,7 +166,7 @@ private List getStats() throws Exception { for (String topic : topics) { 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); @@ -176,8 +176,7 @@ private List getStats() 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); } else { committedOffset = committedMessage.getOffset(); committedTimestampMillis = getTimestamp(committedMessage); @@ -185,7 +184,7 @@ private List getStats() throws Exception { 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 = getTimestamp(lastMessage); @@ -203,10 +202,9 @@ private List getStats() throws Exception { stats.add(Stat.createInstance("secor.lag.offsets", tags, Long.toString(offsetLag), timestamp)); stats.add(Stat.createInstance("secor.lag.seconds", tags, Long.toString(timestampMillisLag / 1000), timestamp)); - LOG.debug("topic " + topic + " partition " + partition + " committed offset " + - committedOffset + " last offset " + lastOffset + " committed timestamp " + - (committedTimestampMillis / 1000) + " last timestamp " + - (lastTimestampMillis / 1000)); + LOG.debug("topic {} partition {} committed offset {} last offset {} committed timestamp {} last timestamp {}", + topic, partition, committedOffset, lastOffset, + (committedTimestampMillis / 1000), (lastTimestampMillis / 1000)); } } } diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index fdfa58711..7a53b6d5d 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -76,7 +76,7 @@ private Future upload(LogFilePath localPath) throws Exception { localPath.getExtension()); final String localLogFilename = localPath.getLogFilePath(); final String s3LogFilename = s3Path.getLogFilePath(); - LOG.info("uploading file " + localLogFilename + " to " + s3LogFilename); + LOG.info("uploading file {} to {}", localLogFilename, s3LogFilename); return executor.submit(new Runnable() { @Override public void run() { @@ -110,8 +110,7 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { long zookeeperComittedOffsetCount = mZookeeperConnector.getCommittedOffsetCount( topicPartition); if (zookeeperComittedOffsetCount == committedOffsetCount) { - LOG.info("uploading topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition()); + LOG.info("uploading topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); Collection paths = mFileRegistry.getPaths(topicPartition); List> uploadFutures = new ArrayList>(); for (LogFilePath path : paths) { @@ -183,10 +182,10 @@ 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 " + srcPath.getLogFilePath() + " to " + - dstPath.getLogFilePath() + " with start offset " + startOffset); + LOG.info("trimmed {} messages from {} to {} with start offset {}", + copiedMessages, srcPath.getLogFilePath(), dstPath.getLogFilePath(), startOffset); } } @@ -209,19 +208,15 @@ private void checkTopicPartition(TopicPartition topicPartition) throws Exception if (oldOffsetCount == newOffsetCount) { uploadFiles(topicPartition); } else if (newOffsetCount > lastSeenOffset) { // && oldOffset < newOffset - LOG.debug("last seen offset " + lastSeenOffset + - " is lower than committed offset count " + newOffsetCount + - ". Deleting files in topic " + topicPartition.getTopic() + - " partition " + topicPartition.getPartition()); + 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); diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index d5d90e3e9..12dd1109d 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -70,10 +70,10 @@ public void adjustOffset(Message message) throws IOException { 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()); From 80d14e6b528fcf34f4bd39bfd5ea8dce0f303127 Mon Sep 17 00:00:00 2001 From: yejun Date: Thu, 2 Jul 2015 11:34:03 -0700 Subject: [PATCH 064/330] fix a rare race condition --- src/main/java/com/pinterest/secor/uploader/Uploader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index 7a53b6d5d..d98294766 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -102,8 +102,6 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { topicPartition.getTopic(), topicPartition.getPartition()); - // Deleting writers closes their streams flushing all pending data to the disk. - mFileRegistry.deleteWriters(topicPartition); mZookeeperConnector.lock(lockPath); try { // Check if the committed offset has changed. @@ -111,6 +109,8 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { topicPartition); if (zookeeperComittedOffsetCount == 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> uploadFutures = new ArrayList>(); for (LogFilePath path : paths) { From bec77cde670419b7281adafe4dbd63364ec9a958 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Mon, 13 Jul 2015 19:10:19 -0700 Subject: [PATCH 065/330] Adding support for hourly s3 data ingestion from secor: Summary: - Add the capability of partition the data file in S3 using hourly folder: s3://pinlogs/secor_raw_logs/topic1/dt=2015-07-07/hr=05/ This way, the data file will be on S3 much sooner (hourly vs daily), people can check the result on S3 much sooner and this also opens the door to have an faster hourly data pipeline on Hadoop side as well. The hr folder values are from 00-23 - To trigger the hourly partition and finalization, add the following parameter in your secor config: * message.timestamp.using.hour=true And change the upload threshold to less than one hour: * secor.max.file.age.seconds=3000 - Change the hive partition registration code to register partition using both dt and hr column (it does require the HIVE table to be created or altered to have both dt and hr as the partition columns) - The enhancements are done through the following: * Change the TimestampedMessageParser to allow it to extract both the dt and hr from the timestamp field * Change the Finalizer to go through both the hr and dt folder to generate the SUCCESS file and do hive registration - the dt finalizer lag before finalization was old way: 1 day + 1 hour lag - the hr finalizer lag was defined as: 1 hour lag - Added more unit test on message parser on hourly behavior - Added more E2E tests to cover the partitioner and hourly ingestion Test Plan: Added both unit tests and e2e tests, and tested manually for a new topic on S3 --- .../config/secor.dev.hr.partition.properties | 11 ++ src/main/config/secor.dev.properties | 4 +- .../pinterest/secor/common/SecorConfig.java | 12 ++ .../main/TestLogMessageProducerMain.java | 21 +- .../secor/parser/PartitionFinalizer.java | 91 +++++++-- .../parser/TimestampedMessageParser.java | 20 +- .../secor/tools/TestLogMessageProducer.java | 16 +- .../pinterest/secor/uploader/Uploader.java | 2 + src/main/scripts/run_tests.sh | 179 +++++++++++++++--- .../secor/parser/JsonMessageParserTest.java | 20 ++ 10 files changed, 326 insertions(+), 50 deletions(-) create mode 100644 src/main/config/secor.dev.hr.partition.properties 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..2b190ae37 --- /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 + +message.timestamp.using.hour=true + +ostrich.port=9998 diff --git a/src/main/config/secor.dev.properties b/src/main/config/secor.dev.properties index b1ad32099..ec30afa71 100644 --- a/src/main/config/secor.dev.properties +++ b/src/main/config/secor.dev.properties @@ -19,6 +19,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/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 2e03856f2..d3fe22147 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -211,6 +211,10 @@ public String getMessageTimestampInputPattern() { return getString("message.timestamp.input.pattern"); } + public boolean getMessageTimestampUsingHour() { + return getBoolean("message.timestamp.using.hour", false); + } + public String getHivePrefix() { return getString("secor.hive.prefix"); } @@ -252,6 +256,14 @@ private int getInt(String name) { return mProperties.getInt(name); } + private boolean getBoolean(String name, boolean required) { + if (required) { + checkProperty(name); + return mProperties.getBoolean(name); + } + return mProperties.getBoolean(name, false); + } + private long getLong(String name) { return mProperties.getLong(name); } diff --git a/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java b/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java index 6e768c3df..25c3a86ee 100644 --- a/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java +++ b/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java @@ -63,6 +63,19 @@ private static CommandLine parseArgs(String[] args) throws ParseException { .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); @@ -74,9 +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, type); + TestLogMessageProducer producer = new TestLogMessageProducer( + topic, messages, type, broker, timeshift); producer.start(); } } catch (Throwable t) { @@ -84,4 +101,4 @@ public static void main(String[] args) { System.exit(1); } } -} \ No newline at end of file +} diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index b826db3ae..fd85bbf3b 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -47,6 +47,7 @@ public class PartitionFinalizer { private KafkaClient mKafkaClient; private QuboleClient mQuboleClient; private String mFileExtension; + private final boolean usingHourly; public PartitionFinalizer(SecorConfig config) throws Exception { mConfig = config; @@ -61,6 +62,7 @@ public PartitionFinalizer(SecorConfig config) throws Exception { } else { mFileExtension = ""; } + usingHourly = config.getMessageTimestampUsingHour(); } private long getLastTimestampMillis(TopicPartition topicPartition) throws Exception { @@ -120,21 +122,35 @@ private long getCommittedTimestampMillis(String topic) throws Exception { private NavigableSet getPartitions(String topic) throws IOException, ParseException { final String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); - String[] partitions = {"dt="}; + String[] partitions = usingHourly ? new String[]{"dt=", "hr="} : new String[]{"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)$"); + if (usingHourly) { + List dirs = new ArrayList(); + for (String partionDir : partitionDirs) { + dirs.addAll(Arrays.asList(FileUtil.list(partionDir))); + } + partitionDirs = dirs.toArray(new String[dirs.size()]); + } + String patternStr = ".*/dt=(\\d\\d\\d\\d-\\d\\d-\\d\\d)"; + if (usingHourly) { + patternStr += "/hr=(\\d\\d)"; + } + patternStr += "$"; + LOG.info("patternStr: " + patternStr); + Pattern pattern = Pattern.compile(patternStr); 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"); + String hour = usingHourly ? matcher.group(2) : "00"; + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd-HH"); format.setTimeZone(TimeZone.getTimeZone("UTC")); Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); - calendar.setTime(format.parse(date)); + calendar.setTime(format.parse(date + "-" + hour)); result.add(calendar); } } @@ -146,27 +162,74 @@ private void finalizePartitionsUpTo(String topic, Calendar calendar) throws IOEx 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")); + SimpleDateFormat dtFormat = new SimpleDateFormat("yyyy-MM-dd"); + SimpleDateFormat hrFormat = new SimpleDateFormat("HH"); + dtFormat.setTimeZone(TimeZone.getTimeZone("UTC")); + hrFormat.setTimeZone(TimeZone.getTimeZone("UTC")); + NavigableSet finishedDates = new TreeSet(); for (Calendar partition : partitionDates) { - String partitionStr = format.format(partition.getTime()); - String[] partitions = {"dt=" + partitionStr}; + String dtPartitionStr = dtFormat.format(partition.getTime()); + String hrPartitionStr = hrFormat.format(partition.getTime()); + String[] partitions = usingHourly + ? new String[]{"dt=" + dtPartitionStr, "hr=" + hrPartitionStr} + : new String[]{"dt=" + dtPartitionStr}; 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; + LOG.info("File exist already, short circuit return. " + successFilePath); + break; } try { - mQuboleClient.addPartition(mConfig.getHivePrefix() + topic, "dt='" + partitionStr + "'"); + String parStr = "dt='" + dtPartitionStr + "'"; + if (usingHourly) { + parStr += ", hr='" + hrPartitionStr + "'"; + } + String hivePrefix = null; + try { + hivePrefix = mConfig.getHivePrefix(); + } catch (RuntimeException ex) { + LOG.warn("HivePrefix is not defined. Skip hive registration"); + } + if (hivePrefix != null) { + mQuboleClient.addPartition(hivePrefix + topic, parStr); + } } catch (Exception e) { - LOG.error("failed to finalize topic {} partition dt={}", topic , partitionStr, e); + LOG.error("failed to finalize topic " + topic + + " partition dt=" + dtPartitionStr + " hr=" + hrPartitionStr, + e); continue; } LOG.info("touching file {}", successFilePath); FileUtil.touch(successFilePath); + + + // We need to mark the successFile for the dt folder as well + if (usingHourly) { + Calendar yesterday = Calendar.getInstance(TimeZone.getTimeZone("UTC")); + yesterday.setTimeInMillis(partition.getTimeInMillis()); + yesterday.add(Calendar.DAY_OF_MONTH, -1); + finishedDates.add(yesterday); + } + } + + // Reverse order to enable short circuit return + finishedDates = finishedDates.descendingSet(); + for (Calendar partition : finishedDates) { + String dtPartitionStr = dtFormat.format(partition.getTime()); + String[] partitions = new String[]{"dt=" + dtPartitionStr}; + LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, partitions, + mConfig.getGeneration(), 0, 0, mFileExtension); + String logFileDir = logFilePath.getLogFileDir(); + String successFilePath = logFileDir + "/_SUCCESS"; + if (FileUtil.exists(successFilePath)) { + LOG.info("File exist already, short circuit return. " + successFilePath); + break; + } + LOG.info("touching file " + successFilePath); + FileUtil.touch(successFilePath); } } @@ -223,9 +286,11 @@ public void finalizePartitions() throws Exception { if (finalizedTimestampMillis != -1) { Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); calendar.setTimeInMillis(finalizedTimestampMillis); - // Introduce a lag of one day and one hour. + if (!usingHourly) { + calendar.add(Calendar.DAY_OF_MONTH, -1); + } + // Introduce a lag of one hour. calendar.add(Calendar.HOUR, -1); - calendar.add(Calendar.DAY_OF_MONTH, -1); finalizePartitionsUpTo(topic, calendar); } } diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 74e4b0a02..ff45295a1 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -25,12 +25,17 @@ public abstract class TimestampedMessageParser extends MessageParser { - private SimpleDateFormat mFormatter; + private final SimpleDateFormat dtFormatter; + private final SimpleDateFormat hrFormatter; + private final boolean usingHourly; public TimestampedMessageParser(SecorConfig config) { super(config); - mFormatter = new SimpleDateFormat("yyyy-MM-dd"); - mFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + dtFormatter = new SimpleDateFormat("yyyy-MM-dd"); + dtFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + hrFormatter = new SimpleDateFormat("HH"); + hrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + usingHourly = config.getMessageTimestampUsingHour(); } public abstract long extractTimestampMillis(final Message message) throws Exception; @@ -54,7 +59,12 @@ public String[] extractPartitions(Message message) throws Exception { // Date constructor takes milliseconds since epoch. long timestampMillis = extractTimestampMillis(message); Date date = new Date(timestampMillis); - String result[] = {"dt=" + mFormatter.format(date)}; - return result; + String dt = "dt=" + dtFormatter.format(date); + String hr = "hr=" + hrFormatter.format(date); + if (usingHourly) { + return new String[]{dt, hr}; + } else { + return new String[]{dt}; + } } } diff --git a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java index fee7070cd..a8eb86023 100644 --- a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java +++ b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java @@ -40,16 +40,25 @@ public class TestLogMessageProducer extends Thread { 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, String type) { + 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"); @@ -69,7 +78,8 @@ public void run() { 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); diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index d98294766..65cb1e617 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -199,6 +199,7 @@ private void trimFiles(TopicPartition topicPartition, long startOffset) throws E private 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()) { long newOffsetCount = mZookeeperConnector.getCommittedOffsetCount(topicPartition); @@ -206,6 +207,7 @@ private void checkTopicPartition(TopicPartition topicPartition) throws Exception 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 {} is lower than committed offset count {}. Deleting files in topic {} partition {}", diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 6a021ee32..0e032d96c 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -38,7 +38,7 @@ PARENT_DIR=/tmp/secor_dev LOGS_DIR=${PARENT_DIR}/logs BUCKET=${SECOR_BUCKET:-test-bucket} S3_LOGS_DIR=s3://${BUCKET}/secor_dev -MESSAGES=1000 +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 @@ -46,13 +46,19 @@ HADOOP_NATIVE_LIB_PATH=lib ADDITIONAL_OPTS= # various reader writer options to be used for testing -declare -A READER_WRITERS -READER_WRITERS[json]=com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory -READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory +# +# However older bash (ver <= 3) does not support associative array, falls back +# to use two arrays +# declare -A READER_WRITERS +# +declare -a READER_WRITER_KEYS +READER_WRITER_KEYS=(json binary) +declare -a READER_WRITERS +READER_WRITERS=(com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory) # 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=${SECOR_WAIT_TIME:-120} +WAIT_TIME=${SECOR_WAIT_TIME:-30} BASE_DIR=$(dirname $0) CONF_DIR=${BASE_DIR}/.. @@ -73,7 +79,7 @@ check_for_native_libs() { recreate_dirs() { run_command "rm -r -f ${PARENT_DIR}" - if [ -n ${SECOR_LOCAL_S3} ]; then + if [ -n "${SECOR_LOCAL_S3}" ]; then run_command "s3cmd -c ${CONF_DIR}/test.s3cfg ls ${S3_LOGS_DIR} | awk '{ print \$4 }' | xargs -L 1 s3cmd -c ${CONF_DIR}/test.s3cfg del" else run_command "s3cmd del --recursive ${S3_LOGS_DIR}" @@ -85,7 +91,7 @@ recreate_dirs() { } start_s3() { - if [ -n ${SECOR_LOCAL_S3} ]; then + 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 2 @@ -97,7 +103,7 @@ start_s3() { } stop_s3() { - if [ -n ${SECOR_LOCAL_S3} ]; then + if [ -n "${SECOR_LOCAL_S3}" ]; then run_command "pkill -9 'fakes3' > /dev/null 2>&1 || true" run_command "rm -r -f /tmp/fakes3" fi @@ -137,23 +143,46 @@ 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 secor-0.1-SNAPSHOT.jar:lib/* \ + 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 \ 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.test.backup.properties -cp ${CLASSPATH} \ - com.pinterest.secor.main.TestLogMessageProducerMain -t test -m $1 -p 1 -type ${MESSAGE_TYPE} > \ + -Dconfig=secor.test.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + 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() { + 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 \ @@ -169,6 +198,21 @@ verify() { stop_s3 exit ${VERIFICATION_EXIT_CODE} fi + + # Verify SUCCESS file + if [ -n "${SECOR_LOCAL_S3}" ]; then + run_command "s3cmd ls -c ${CONF_DIR}/test.s3cfg -r ${S3_LOGS_DIR} | gr +ep _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 + count=$( Date: Wed, 15 Jul 2015 15:35:30 +0200 Subject: [PATCH 066/330] fix bad logging message --- src/main/java/com/pinterest/secor/common/OffsetTracker.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/OffsetTracker.java b/src/main/java/com/pinterest/secor/common/OffsetTracker.java index 8fff6eb70..4a390ca84 100644 --- a/src/main/java/com/pinterest/secor/common/OffsetTracker.java +++ b/src/main/java/com/pinterest/secor/common/OffsetTracker.java @@ -52,10 +52,10 @@ public long setLastSeenOffset(TopicPartition topicPartition, long offset) { mLastSeenOffset.put(topicPartition, offset); if (lastSeenOffset + 1 != offset) { if (lastSeenOffset >= 0) { - LOG.warn("offset for topic {} partition {} changed from {} to {}", + 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 {}", + LOG.info("starting to consume topic {} partition {} from offset {}", topicPartition.getTopic(),topicPartition.getPartition(),offset); } } From 7632f4856cbf2fa8455007a4b8ddfef62d45ffe0 Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Wed, 15 Jul 2015 11:16:49 -0500 Subject: [PATCH 067/330] Simple timestamped message parser for protocol buffers. --- pom.xml | 5 ++ .../secor/parser/ProtobufMessageParser.java | 49 ++++++++++++++++++ .../parser/ProtobufMessageParserTest.java | 51 +++++++++++++++++++ 3 files changed, 105 insertions(+) create mode 100644 src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java create mode 100644 src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java diff --git a/pom.xml b/pom.xml index 069085186..dae24e2e9 100644 --- a/pom.xml +++ b/pom.xml @@ -48,6 +48,11 @@ + + com.google.protobuf + protobuf-java + 2.6.1 + net.java.dev.jets3t jets3t 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..03c3b0b93 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.parser; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import com.google.protobuf.CodedInputStream; + +import java.io.IOException; + +/** + * Basic protocol buffer parser. + * + * Assumes that the timestamp field is the first field, is required, + * and is a uint64. A more advanced parser might support an arbitrary + * field number (non-nested to keep things simple) and perhaps + * different data types. + * + * @author Liam Stewart (liam.stewart@gmail.com) + */ +public class ProtobufMessageParser extends TimestampedMessageParser { + public ProtobufMessageParser(SecorConfig config) { + super(config); + } + + @Override + public long extractTimestampMillis(final Message message) throws IOException { + CodedInputStream input = CodedInputStream.newInstance(message.getPayload()); + + // Don't really care about the tag, but need to read it to get + // to the payload. + int tag = input.readTag(); + return toMillis(input.readUInt64()); + } +} 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..14a0288ca --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.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 com.google.protobuf.CodedOutputStream; + +@RunWith(PowerMockRunner.class) +public class ProtobufMessageParserTest extends TestCase { + private SecorConfig mConfig; + + 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, data); + } + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + } + + @Test + public void testExtractTimestampMillis() throws Exception { + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + assertEquals(1405970352000l, parser.extractTimestampMillis(buildMessage(1405970352l))); + assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); + } +} From 42ab0a61ea1a4353b614cb2ff37638fc1893d658 Mon Sep 17 00:00:00 2001 From: Yejun Yang Date: Tue, 12 May 2015 11:41:09 -0700 Subject: [PATCH 068/330] update kafka version fix test for newer kafka fix test script use roundbin assignment --- pom.xml | 2 +- .../pinterest/secor/reader/MessageReader.java | 1 + src/main/scripts/run_tests.sh | 25 ++++++++----------- src/main/scripts/run_zookeeper_command.sh | 5 ---- 4 files changed, 13 insertions(+), 20 deletions(-) diff --git a/pom.xml b/pom.xml index 069085186..963f7a091 100644 --- a/pom.xml +++ b/pom.xml @@ -56,7 +56,7 @@ org.apache.kafka kafka_2.10 - 0.8.1.1 + 0.8.2.1 org.slf4j diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 245320346..11ff76214 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -110,6 +110,7 @@ private ConsumerConfig createConsumerConfig() throws UnknownHostException { props.put("auto.offset.reset", "smallest"); props.put("consumer.timeout.ms", Integer.toString(mConfig.getConsumerTimeoutMs())); props.put("consumer.id", IdUtil.getConsumerId()); + props.put("partition.assignment.strategy", "roundrobin"); if (mConfig.getRebalanceMaxRetries() != null && !mConfig.getRebalanceMaxRetries().isEmpty()) { props.put("rebalance.max.retries", mConfig.getRebalanceMaxRetries()); diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 6a021ee32..8034fb088 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -73,7 +73,7 @@ check_for_native_libs() { recreate_dirs() { run_command "rm -r -f ${PARENT_DIR}" - if [ -n ${SECOR_LOCAL_S3} ]; then + if [ -n "${SECOR_LOCAL_S3}" ]; then run_command "s3cmd -c ${CONF_DIR}/test.s3cfg ls ${S3_LOGS_DIR} | awk '{ print \$4 }' | xargs -L 1 s3cmd -c ${CONF_DIR}/test.s3cfg del" else run_command "s3cmd del --recursive ${S3_LOGS_DIR}" @@ -85,7 +85,7 @@ recreate_dirs() { } start_s3() { - if [ -n ${SECOR_LOCAL_S3} ]; then + 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 2 @@ -97,7 +97,7 @@ start_s3() { } stop_s3() { - if [ -n ${SECOR_LOCAL_S3} ]; then + if [ -n "${SECOR_LOCAL_S3}" ]; then run_command "pkill -9 'fakes3' > /dev/null 2>&1 || true" run_command "rm -r -f /tmp/fakes3" fi @@ -175,17 +175,14 @@ verify() { set_offsets_in_zookeeper() { for group in secor_backup secor_partition; do for partition in 0 1; do - run_command "${BASE_DIR}/run_zookeeper_command.sh localhost:2181 create \ - /consumers \'\' > ${LOGS_DIR}/run_zookeeper_command.log 2>&1" - run_command "${BASE_DIR}/run_zookeeper_command.sh localhost:2181 create \ - /consumers/${group} \'\' > ${LOGS_DIR}/run_zookeeper_command.log 2>&1" - run_command "${BASE_DIR}/run_zookeeper_command.sh localhost:2181 create \ - /consumers/${group}/offsets \'\' > ${LOGS_DIR}/run_zookeeper_command.log 2>&1" - run_command "${BASE_DIR}/run_zookeeper_command.sh localhost:2181 create \ - /consumers/${group}/offsets/test \'\' > ${LOGS_DIR}/run_zookeeper_command.log 2>&1" - run_command "${BASE_DIR}/run_zookeeper_command.sh localhost:2181 create \ - /consumers/${group}/offsets/test/${partition} $1 > \ - ${LOGS_DIR}/run_zookeeper_command.log 2>&1" + 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 } diff --git a/src/main/scripts/run_zookeeper_command.sh b/src/main/scripts/run_zookeeper_command.sh index 178b92ada..873737b95 100755 --- a/src/main/scripts/run_zookeeper_command.sh +++ b/src/main/scripts/run_zookeeper_command.sh @@ -17,11 +17,6 @@ # 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 From 89fed7e6379cda2589ef0e778f78c0ddfbbbf24a Mon Sep 17 00:00:00 2001 From: Yejun Yang Date: Mon, 8 Jun 2015 16:38:08 -0700 Subject: [PATCH 069/330] fix a config reload on every message read fix another config reload on every write Only update stats and rate limit every per second remove uncessary function call rename variable and remove unused function --- .../pinterest/secor/reader/MessageReader.java | 16 +++++++++++++--- .../com/pinterest/secor/util/RateLimitUtil.java | 4 ++-- .../pinterest/secor/writer/MessageWriter.java | 6 ++++-- 3 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 245320346..59b768037 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -54,6 +54,9 @@ public class MessageReader { private ConsumerConnector mConsumerConnector; private ConsumerIterator mIterator; private HashMap mLastAccessTime; + private final int mTopicPartitionForgetSeconds; + private final int mCheckMessagesPerSecond; + private int mNMessages; public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws UnknownHostException { @@ -69,6 +72,8 @@ public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws mIterator = stream.iterator(); mLastAccessTime = new HashMap(); StatsUtil.setLabel("secor.kafka.consumer.id", IdUtil.getConsumerId()); + mTopicPartitionForgetSeconds = mConfig.getTopicPartitionForgetSeconds(); + mCheckMessagesPerSecond = mConfig.getMessagesPerSecond() / mConfig.getConsumerThreads(); } private void updateAccessTime(TopicPartition topicPartition) { @@ -78,7 +83,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(); } } @@ -141,7 +146,10 @@ 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(); Message message = new Message(kafkaMessage.topic(), kafkaMessage.partition(), kafkaMessage.offset(), kafkaMessage.message()); @@ -151,7 +159,9 @@ public Message read() { // Skip already committed messages. long committedOffsetCount = mOffsetTracker.getTrueCommittedOffsetCount(topicPartition); LOG.debug("read message {}", message); - exportStats(); + if (mNMessages % mCheckMessagesPerSecond == 0) { + exportStats(); + } if (message.getOffset() < committedOffsetCount) { LOG.debug("skipping message {} because its offset precedes committed offset count {}", message, committedOffsetCount); 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/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index 12dd1109d..ddc4218f8 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -48,6 +48,7 @@ public class MessageWriter { private String mFileExtension; private CompressionCodec mCodec; private String mLocalPrefix; + private final int mGeneration; public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry) throws Exception { @@ -61,6 +62,7 @@ public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, mFileExtension = ""; } mLocalPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); + mGeneration = mConfig.getGeneration(); } public void adjustOffset(Message message) throws IOException { @@ -83,11 +85,11 @@ public void write(ParsedMessage message) throws Exception { TopicPartition topicPartition = new TopicPartition(message.getTopic(), message.getKafkaPartition()); long offset = mOffsetTracker.getAdjustedCommittedOffsetCount(topicPartition); - LogFilePath path = new LogFilePath(mLocalPrefix, mConfig.getGeneration(), offset, message, + LogFilePath path = new LogFilePath(mLocalPrefix, mGeneration, offset, message, mFileExtension); FileWriter writer = mFileRegistry.getOrCreateWriter(path, mCodec); writer.write(new KeyValue(message.getOffset(), message.getPayload())); LOG.debug("appended message {} to file {}. File length {}", - message, path.getLogFilePath(), writer.getLength()); + message, path, writer.getLength()); } } From 9fe9ef2db1d5116a5b171445b6f7596c28443b17 Mon Sep 17 00:00:00 2001 From: lwoessner Date: Thu, 16 Jul 2015 12:56:38 -0600 Subject: [PATCH 070/330] fix null payload bug --- .../com/pinterest/secor/message/Message.java | 28 ++++++++++++++++--- .../pinterest/secor/message/MessageTest.java | 20 +++++++++++++ 2 files changed, 44 insertions(+), 4 deletions(-) create mode 100644 src/test/java/com/pinterest/secor/message/MessageTest.java diff --git a/src/main/java/com/pinterest/secor/message/Message.java b/src/main/java/com/pinterest/secor/message/Message.java index c50ba9444..d348878c8 100644 --- a/src/main/java/com/pinterest/secor/message/Message.java +++ b/src/main/java/com/pinterest/secor/message/Message.java @@ -20,55 +20,75 @@ import java.io.OutputStream; import java.lang.String; + /** * Message represents a raw Kafka log message. * * @author Pawel Garbacki (pawel@pinterest.com) */ public class Message { + + private String mTopic; private int mKafkaPartition; private long mOffset; private byte[] mPayload; + protected String fieldsToString() { - return "topic='" + mTopic + '\'' + - ", kafkaPartition=" + mKafkaPartition + - ", offset=" + mOffset + - ", payload=" + new String(mPayload); + + return "topic='" + mTopic + '\'' + ", kafkaPartition=" + mKafkaPartition + ", offset=" + mOffset + ", payload=" + + new String(mPayload); } + @Override public String toString() { + return "Message{" + fieldsToString() + '}'; } + public Message(String topic, int kafkaPartition, long offset, byte[] payload) { + mTopic = topic; mKafkaPartition = kafkaPartition; mOffset = offset; mPayload = payload; + + if (mPayload == null) { + mPayload = new byte[0]; + } } + public String getTopic() { return mTopic; } + public int getKafkaPartition() { + return mKafkaPartition; } + public long getOffset() { + return mOffset; } + public byte[] getPayload() { + return mPayload; } + public void write(OutputStream output) throws IOException { + output.write(mPayload); } } diff --git a/src/test/java/com/pinterest/secor/message/MessageTest.java b/src/test/java/com/pinterest/secor/message/MessageTest.java new file mode 100644 index 000000000..e49d894e4 --- /dev/null +++ b/src/test/java/com/pinterest/secor/message/MessageTest.java @@ -0,0 +1,20 @@ +package com.pinterest.secor.message; + +import static org.junit.Assert.*; + +import org.junit.Test; + + + +public class MessageTest { + + + @Test + public void testNullPayload() { + Message message = new Message("testTopic", 0, 123, null); + System.out.println(message); + + // no assert necessary, just making sure it does not throw a NullPointerException + } + +} From 05159e8aabe253087afe86c9fb9b0ee43945ff8a Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 16 Jul 2015 16:01:53 -0700 Subject: [PATCH 071/330] Update based on Pawel's feedback. Instead of hard-coding daily/hourly concept in the PartitionFinalizer. Move the concept of partition boundary and partition ordering into a new interface Partitioner. PartitionFinalizer will work with the Partitioner to the last partition to finalize up to and collect all the partitions previous to that partition from partitioner. --- src/main/config/secor.prod.properties | 1 + .../pinterest/secor/common/LogFilePath.java | 14 +- .../pinterest/secor/common/SecorConfig.java | 20 +- .../secor/parser/PartitionFinalizer.java | 326 ++++++------------ .../pinterest/secor/parser/Partitioner.java | 97 ++++++ .../parser/TimestampedMessageParser.java | 113 +++++- src/main/scripts/run_tests.sh | 115 +++--- 7 files changed, 381 insertions(+), 305 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/parser/Partitioner.java diff --git a/src/main/config/secor.prod.properties b/src/main/config/secor.prod.properties index 0801a5c99..56047cb57 100644 --- a/src/main/config/secor.prod.properties +++ b/src/main/config/secor.prod.properties @@ -37,5 +37,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/java/com/pinterest/secor/common/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java index 367dc03f4..753288c0d 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -42,13 +42,13 @@ * @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 mKafkaPartition; + private final long mOffset; + private final String mExtension; public LogFilePath(String prefix, int generation, long lastCommittedOffset, ParsedMessage message, String extension) { diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index d3fe22147..7d66ddd61 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -215,6 +215,14 @@ public boolean getMessageTimestampUsingHour() { return getBoolean("message.timestamp.using.hour", false); } + public int getFinalizerLagSecond() { + return getInt("finalizer.lag.second", 3600); + } + + public int getFinalizerLookbackPeriods() { + return getInt("finalizer.lookback.periods", 10); + } + public String getHivePrefix() { return getString("secor.hive.prefix"); } @@ -256,12 +264,12 @@ private int getInt(String name) { return mProperties.getInt(name); } - private boolean getBoolean(String name, boolean required) { - if (required) { - checkProperty(name); - return mProperties.getBoolean(name); - } - return mProperties.getBoolean(name, false); + private int getInt(String name, int defaultValue) { + return mProperties.getInt(name, defaultValue); + } + + private boolean getBoolean(String name, boolean defaultValue) { + return mProperties.getBoolean(name, defaultValue); } private long getLong(String name) { diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index fd85bbf3b..b9297d589 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -25,12 +25,10 @@ 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,13 +39,13 @@ public class PartitionFinalizer { private static final Logger LOG = LoggerFactory.getLogger(PartitionFinalizer.class); - private SecorConfig mConfig; - private ZookeeperConnector mZookeeperConnector; - private TimestampedMessageParser mMessageParser; - private KafkaClient mKafkaClient; - private QuboleClient mQuboleClient; - private String mFileExtension; - private final boolean usingHourly; + 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; @@ -62,216 +60,123 @@ public PartitionFinalizer(SecorConfig config) throws Exception { } else { mFileExtension = ""; } - usingHourly = config.getMessageTimestampUsingHour(); + mLookbackPeriods = config.getFinalizerLookbackPeriods(); + LOG.info("Lookback periods: " + mLookbackPeriods); } - private long getLastTimestampMillis(TopicPartition topicPartition) throws Exception { - Message message = mKafkaClient.getLastMessage(topicPartition); - if (message == null) { - // This will happen if no messages have been posted to the given topic partition. - LOG.error("No message found for topic {} partition {}" + topicPartition.getTopic(), topicPartition.getPartition()); - return -1; - } - return mMessageParser.extractTimestampMillis(message); - } - - private long getLastTimestampMillis(String topic) throws Exception { - final int numPartitions = mKafkaClient.getNumPartitions(topic); - long max_timestamp = Long.MIN_VALUE; - for (int partition = 0; partition < numPartitions; ++partition) { - TopicPartition topicPartition = new TopicPartition(topic, partition); - long timestamp = getLastTimestampMillis(topicPartition); - if (timestamp > max_timestamp) { - max_timestamp = timestamp; - } - } - if (max_timestamp == Long.MIN_VALUE) { - return -1; - } - return max_timestamp; - } - - private long getCommittedTimestampMillis(TopicPartition topicPartition) throws Exception { - Message message = mKafkaClient.getCommittedMessage(topicPartition); - if (message == null) { - LOG.error("No message found for topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); - return -1; - } - return mMessageParser.extractTimestampMillis(message); - } - - private long getCommittedTimestampMillis(String topic) throws Exception { + private String[] getFinalizedUptoPartitions(String topic) throws Exception { final int numPartitions = mKafkaClient.getNumPartitions(topic); - long minTimestamp = Long.MAX_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 = getCommittedTimestampMillis(topicPartition); - if (timestamp == -1) { - return -1; - } else { - if (timestamp < minTimestamp) { - minTimestamp = 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: {}, commmitted: {}", + topicPartition.getTopic(), topicPartition.getPartition(), + lastMessage, committedMessage); + continue; } + lastMessages.add(lastMessage); + committedMessages.add(committedMessage); } - if (minTimestamp == Long.MAX_VALUE) { - return -1; - } - return minTimestamp; + return mMessageParser.getFinalizedUptoPartitions(lastMessages, committedMessages); } - private NavigableSet getPartitions(String topic) throws IOException, ParseException { + private void finalizePartitionsUpTo(String topic, String[] partitions) throws Exception { final String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); - String[] partitions = usingHourly ? new String[]{"dt=", "hr="} : new String[]{"dt="}; - LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, partitions, - mConfig.getGeneration(), 0, 0, mFileExtension); - String parentDir = logFilePath.getLogFileParentDir(); - String[] partitionDirs = FileUtil.list(parentDir); - if (usingHourly) { - List dirs = new ArrayList(); - for (String partionDir : partitionDirs) { - dirs.addAll(Arrays.asList(FileUtil.list(partionDir))); - } - partitionDirs = dirs.toArray(new String[dirs.size()]); - } - String patternStr = ".*/dt=(\\d\\d\\d\\d-\\d\\d-\\d\\d)"; - if (usingHourly) { - patternStr += "/hr=(\\d\\d)"; - } - patternStr += "$"; - LOG.info("patternStr: " + patternStr); - Pattern pattern = Pattern.compile(patternStr); - TreeSet result = new TreeSet(); - for (String partitionDir : partitionDirs) { - Matcher matcher = pattern.matcher(partitionDir); - if (matcher.find()) { - String date = matcher.group(1); - String hour = usingHourly ? matcher.group(2) : "00"; - SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd-HH"); - format.setTimeZone(TimeZone.getTimeZone("UTC")); - Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); - calendar.setTime(format.parse(date + "-" + hour)); - result.add(calendar); - } - } - return result; - } - private void finalizePartitionsUpTo(String topic, Calendar calendar) throws IOException, - ParseException, InterruptedException { - NavigableSet partitionDates = - getPartitions(topic).headSet(calendar, true).descendingSet(); - final String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); - SimpleDateFormat dtFormat = new SimpleDateFormat("yyyy-MM-dd"); - SimpleDateFormat hrFormat = new SimpleDateFormat("HH"); - dtFormat.setTimeZone(TimeZone.getTimeZone("UTC")); - hrFormat.setTimeZone(TimeZone.getTimeZone("UTC")); - NavigableSet finishedDates = new TreeSet(); - for (Calendar partition : partitionDates) { - String dtPartitionStr = dtFormat.format(partition.getTime()); - String hrPartitionStr = hrFormat.format(partition.getTime()); - String[] partitions = usingHourly - ? new String[]{"dt=" + dtPartitionStr, "hr=" + hrPartitionStr} - : new String[]{"dt=" + dtPartitionStr}; - 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)) { - LOG.info("File exist already, short circuit return. " + successFilePath); - break; - } - try { - String parStr = "dt='" + dtPartitionStr + "'"; - if (usingHourly) { - parStr += ", hr='" + hrPartitionStr + "'"; - } - String hivePrefix = null; - try { - hivePrefix = mConfig.getHivePrefix(); - } catch (RuntimeException ex) { - LOG.warn("HivePrefix is not defined. Skip hive registration"); - } - if (hivePrefix != null) { - mQuboleClient.addPartition(hivePrefix + topic, parStr); + // Walk through each dimension of the partitions array + // First finalize the hourly partitions, and then move on to daily partitions + for (int dim = partitions.length; dim > 0; dim--) { + String[] uptoPartitions = Arrays.copyOf(partitions, dim); + LOG.info("Finalize up to (but not include) {} for dim: {}", uptoPartitions, dim); + + 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(s3Prefix, topic, previous, + mConfig.getGeneration(), 0, 0, mFileExtension); + String logFileDir = logFilePath.getLogFileDir(); + 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); } - } catch (Exception e) { - LOG.error("failed to finalize topic " + topic - + " partition dt=" + dtPartitionStr + " hr=" + hrPartitionStr, - e); - continue; + previous = mMessageParser.getPreviousPartitions(previous); } - LOG.info("touching file {}", successFilePath); - FileUtil.touch(successFilePath); - - // We need to mark the successFile for the dt folder as well - if (usingHourly) { - Calendar yesterday = Calendar.getInstance(TimeZone.getTimeZone("UTC")); - yesterday.setTimeInMillis(partition.getTimeInMillis()); - yesterday.add(Calendar.DAY_OF_MONTH, -1); - finishedDates.add(yesterday); + LOG.info("To be finalized partitions: {}", toBeFinalized); + if (toBeFinalized.isEmpty()) { + LOG.warn("There is no partitions to be finalized for dim: {}", dim); + continue; } - } - // Reverse order to enable short circuit return - finishedDates = finishedDates.descendingSet(); - for (Calendar partition : finishedDates) { - String dtPartitionStr = dtFormat.format(partition.getTime()); - String[] partitions = new String[]{"dt=" + dtPartitionStr}; - LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, partitions, - mConfig.getGeneration(), 0, 0, mFileExtension); - String logFileDir = logFilePath.getLogFileDir(); - String successFilePath = logFileDir + "/_SUCCESS"; - if (FileUtil.exists(successFilePath)) { - LOG.info("File exist already, short circuit return. " + successFilePath); - break; - } - LOG.info("touching file " + successFilePath); - FileUtil.touch(successFilePath); - } - } + // 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 (dim == partitions.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 hivePrefix = null; + try { + hivePrefix = mConfig.getHivePrefix(); + } catch (RuntimeException ex) { + LOG.warn("HivePrefix is not defined. Skip hive registration"); + } + if (hivePrefix != null) { + mQuboleClient.addPartition(hivePrefix + topic, sb.toString()); + } + } catch (Exception e) { + LOG.error("failed to finalize topic " + topic, e); + continue; + } + } - /** - * Get finalized timestamp for a given topic partition. Finalized timestamp is the current time - * if the last offset for that topic partition has been committed earlier than an hour ago. - * Otherwise, finalized timestamp is the committed timestamp. - * - * @param topicPartition The topic partition for which we want to compute the finalized - * timestamp. - * @return The finalized timestamp for the topic partition. - * @throws Exception - */ - private long getFinalizedTimestampMillis(TopicPartition topicPartition) throws Exception { - long lastTimestamp = getLastTimestampMillis(topicPartition); - long committedTimestamp = getCommittedTimestampMillis(topicPartition); - long now = System.currentTimeMillis(); - if (lastTimestamp == committedTimestamp && (now - lastTimestamp) > 3600 * 1000) { - return now; - } - return committedTimestamp; - } + // Generate the SUCCESS file at the end + LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, current, + mConfig.getGeneration(), 0, 0, mFileExtension); + String logFileDir = logFilePath.getLogFileDir(); + String successFilePath = logFileDir + "/_SUCCESS"; - 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 {} partition {} is {}", topic, partition, timestamp); - if (timestamp == -1) { - return -1; - } else { - if (timestamp < minTimestamp) { - minTimestamp = timestamp; - } + LOG.info("touching file {}", successFilePath); + FileUtil.touch(successFilePath); } } - if (minTimestamp == Long.MAX_VALUE) { - return -1; - } - return minTimestamp; } public void finalizePartitions() throws Exception { @@ -281,17 +186,10 @@ public void finalizePartitions() throws Exception { LOG.info("skipping topic {}", topic); } else { LOG.info("finalizing topic {}", topic); - long finalizedTimestampMillis = getFinalizedTimestampMillis(topic); - LOG.info("finalized timestamp for topic {} is {}", topic , finalizedTimestampMillis); - if (finalizedTimestampMillis != -1) { - Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); - calendar.setTimeInMillis(finalizedTimestampMillis); - if (!usingHourly) { - calendar.add(Calendar.DAY_OF_MONTH, -1); - } - // Introduce a lag of one hour. - calendar.add(Calendar.HOUR, -1); - finalizePartitionsUpTo(topic, calendar); + 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/Partitioner.java b/src/main/java/com/pinterest/secor/parser/Partitioner.java new file mode 100644 index 000000000..d7a4f78e9 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/Partitioner.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.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-dimentional (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"] + * + * @param partition + * @return + */ + String[] getPreviousPartitions(String[] partition) throws Exception; +} diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index ff45295a1..a13a252db 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -19,23 +19,36 @@ import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; import java.text.SimpleDateFormat; import java.util.Date; import java.util.TimeZone; -public abstract class TimestampedMessageParser extends MessageParser { +public abstract class TimestampedMessageParser extends MessageParser implements Partitioner { + + private static final Logger LOG = LoggerFactory.getLogger(TimestampedMessageParser.class); + + private static final SimpleDateFormat mDtFormatter = new SimpleDateFormat("yyyy-MM-dd"); + private static final SimpleDateFormat mHrFormatter = new SimpleDateFormat("HH"); + private static final SimpleDateFormat mDtHrFormatter = new SimpleDateFormat("yyyy-MM-dd-HH"); + + static { + mDtFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + mHrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + mDtHrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + } - private final SimpleDateFormat dtFormatter; - private final SimpleDateFormat hrFormatter; - private final boolean usingHourly; + private final boolean mUsingHourly; + private final long mLagInSeconds; public TimestampedMessageParser(SecorConfig config) { super(config); - dtFormatter = new SimpleDateFormat("yyyy-MM-dd"); - dtFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); - hrFormatter = new SimpleDateFormat("HH"); - hrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); - usingHourly = config.getMessageTimestampUsingHour(); + mUsingHourly = config.getMessageTimestampUsingHour(); + mLagInSeconds = config.getFinalizerLagSecond(); + LOG.info("UsingHourly: {}, lagInSeconds: {} ", mUsingHourly, mLagInSeconds); } public abstract long extractTimestampMillis(final Message message) throws Exception; @@ -54,17 +67,87 @@ protected static long toMillis(final long timestamp) { return timestampMillis; } + protected String[] generatePartitions(long timestampMillis, boolean usingHourly) + throws Exception { + Date date = new Date(timestampMillis); + String dt = "dt=" + mDtFormatter.format(date); + String hr = "hr=" + mHrFormatter.format(date); + 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 value = dtValue + "-" + hrValue; + Date date = mDtHrFormatter.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); - String dt = "dt=" + dtFormatter.format(date); - String hr = "hr=" + hrFormatter.format(date); - if (usingHourly) { - return new String[]{dt, hr}; + return generatePartitions(timestampMillis, mUsingHourly); + } + + private long getFinalizedTimestampMillis(Message lastMessage, + Message committedMessage) throws Exception { + long lastTimestamp = extractTimestampMillis(lastMessage); + long committedTimestamp = extractTimestampMillis(committedMessage); + long now = System.currentTimeMillis(); + if (lastTimestamp == committedTimestamp && (now - lastTimestamp) > 3600 * 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) { + 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 + long lag = mLagInSeconds * 1000L; + LOG.info("Originally: {}, adjust down {}", minMillis, lag); + return generatePartitions(minMillis - lag, mUsingHourly); + } + + @Override + public String[] getPreviousPartitions(String[] partitions) throws Exception { + long millis = parsePartitions(partitions); + long delta; + if (partitions.length == 1) { + delta = 3600L * 24 * 1000L; + } else if (partitions.length == 2) { + delta = 3600L * 1000L; } else { - return new String[]{dt}; + throw new RuntimeException("Unsupported partitions: " + partitions.length); } + return generatePartitions(millis - delta, partitions.length == 2); } + } diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 0e032d96c..abe6c37b5 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -46,15 +46,11 @@ HADOOP_NATIVE_LIB_PATH=lib ADDITIONAL_OPTS= # various reader writer options to be used for testing +# note associate array needs bash v4 support # -# However older bash (ver <= 3) does not support associative array, falls back -# to use two arrays -# declare -A READER_WRITERS -# -declare -a READER_WRITER_KEYS -READER_WRITER_KEYS=(json binary) -declare -a READER_WRITERS -READER_WRITERS=(com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory) +declare -A READER_WRITERS +READER_WRITERS[json]=com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory +READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory # 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. @@ -80,9 +76,11 @@ check_for_native_libs() { recreate_dirs() { run_command "rm -r -f ${PARENT_DIR}" if [ -n "${SECOR_LOCAL_S3}" ]; then - run_command "s3cmd -c ${CONF_DIR}/test.s3cfg ls ${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} | 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 # create logs directory if [ ! -d ${LOGS_DIR} ]; then @@ -94,7 +92,7 @@ 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 2 + 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`" @@ -104,7 +102,7 @@ start_s3() { stop_s3() { if [ -n "${SECOR_LOCAL_S3}" ]; then - run_command "pkill -9 'fakes3' > /dev/null 2>&1 || true" + run_command "pkill -f 'fakes3' || true" run_command "rm -r -f /tmp/fakes3" fi } @@ -145,7 +143,7 @@ stop_secor() { run_finalizer() { run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.test.partition.properties ${ADDITIONAL_OPTS} -cp secor-0.1-SNAPSHOT.jar:lib/* \ + -Dconfig=secor.test.partition.properties ${ADDITIONAL_OPTS} -cp $CLASSPATH \ com.pinterest.secor.main.PartitionFinalizerMain > ${LOGS_DIR}/finalizer.log 2>&1 " EXIT_CODE=$? @@ -167,7 +165,7 @@ create_topic() { # $2 timeshift in seconds post_messages() { run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.test.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + -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" } @@ -194,15 +192,12 @@ verify() { 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 - stop_all - stop_s3 exit ${VERIFICATION_EXIT_CODE} fi # Verify SUCCESS file if [ -n "${SECOR_LOCAL_S3}" ]; then - run_command "s3cmd ls -c ${CONF_DIR}/test.s3cfg -r ${S3_LOGS_DIR} | gr -ep _SUCCESS | wc -l > /tmp/secor_tests_output.txt" + 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 @@ -275,7 +270,6 @@ post_and_verify_test() { # 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 -# $2: number of messages post_and_finalizer_verify_test() { echo "********************************************************" date=$(date -u +'%Y-%m-%d %H:%M:%S') @@ -291,41 +285,32 @@ post_and_finalizer_verify_test() { return fi + HOUR_TIMESHIFT=$((3600+3600)) + DAY_TIMESHIFT=$((86400+3600)) + + OLD_ADDITIONAL_OPTS=${ADDITIONAL_OPTS} + if [ $1 = "hr" ]; then - ADDITIONAL_OPTS="-Dsecor.file.reader.writer.factory=${READER_WRITERS[${num}]} -Dmessage.timestamp.using.hour=true" - # for hr folder, the finalizer lag is 1 hour - TIMESHIFT=$((3600+1200)) - if [ $2 -ne 0 ]; then - # should be 2 success files, one for hr folder, - # one for yesterday's dt folder - FILES=2 - else - # should be 0 success files - FILES=0 - fi + ADDITIONAL_OPTS="${ADDITIONAL_OPTS} -Dmessage.timestamp.using.hour=true -Dfinalizer.lookback.periods=30" + # should be 2 success files for hr folder, 1 for dt folder + FILES=3 else - ADDITIONAL_OPTS="-Dsecor.file.reader.writer.factory=${READER_WRITERS[${num}]}" - # for dt folder, the finalizer lag is 1 day + 1 hour - TIMESHIFT=$((86400+3600+1200)) - if [ $2 -ne 0 ]; then - # should be 1 success files - FILES=1 - else - # should be 0 success files - FILES=0 - fi + # should be 1 success files for dt folder + FILES=1 fi echo "Expected success file: $FILES" - echo "running post_and_finalizer_verify_test $1 $2" + echo "running post_and_finalizer_verify_test $1" initialize start_secor sleep 3 - # post some older messages - post_messages $2 ${TIMESHIFT} - # post some newer messages + # 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" @@ -334,9 +319,11 @@ post_and_finalizer_verify_test() { echo "start finalizer" run_finalizer - verify $((0+$2+${MESSAGES})) ${FILES} + verify $((${MESSAGES}*3)) ${FILES} stop_all + ADDITIONAL_OPTS=${OLD_ADDITIONAL_OPTS} + echo -e "\e[1;42;97mpost_and_finalizer_verify_test succeeded\e[0m" } @@ -364,18 +351,23 @@ move_offset_back_test() { 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)) 0 set_offsets_in_zookeeper 2 post_messages $((${MESSAGES}*9/10)) 0 - echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" - sleep ${WAIT_TIME} + 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 + ADDITIONAL_OPTS=${OLD_ADDITIONAL_OPTS} + echo -e "\e[1;42;97mmove_offset_back_test succeeded\e[0m" } @@ -385,6 +377,8 @@ post_and_verify_compressed_test() { echo "running post_and_verify_compressed_test" initialize + 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" @@ -396,31 +390,26 @@ post_and_verify_compressed_test() { verify ${MESSAGES} 0 stop_all + ADDITIONAL_OPTS=${OLD_ADDITIONAL_OPTS} + echo -e "\e[1;42;97mpost_and_verify_compressed_test succeeded\e[0m" } check_for_native_libs +stop_s3 start_s3 -# Testing finalizer in partition mode -num=1 -MESSAGE_TYPE=${READER_WRITER_KEYS[${num}]} -echo "********************************************************" -echo "Running hourly finalizer tests for Message Type: ${MESSAGE_TYPE} and ReaderWriter: ${READER_WRITERS[${num}]}" -post_and_finalizer_verify_test hr 0 -post_and_finalizer_verify_test hr 100 - -echo "********************************************************" -echo "Running daily finalizer tests for Message Type: ${MESSAGE_TYPE} and ReaderWriter: ${READER_WRITERS[${num}]}" -post_and_finalizer_verify_test dt 0 -post_and_finalizer_verify_test dt 100 - -for num in 0 ${#READER_WRITERS[@]}-1; do - MESSAGE_TYPE=${READER_WRITER_KEYS[${num}]} - ADDITIONAL_OPTS=-Dsecor.file.reader.writer.factory=${READER_WRITERS[${num}]} +for key in ${!READER_WRITERS[@]}; do + MESSAGE_TYPE=${key} + ADDITIONAL_OPTS=-Dsecor.file.reader.writer.factory=${READER_WRITERS[${key}]} echo "********************************************************" - echo "Running tests for Message Type: ${MESSAGE_TYPE} and ReaderWriter: ${READER_WRITERS[${num}]}" + echo "Running tests for Message Type: ${MESSAGE_TYPE} and ReaderWriter: ${READER_WRITERS[${key}]}" 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 From 32f19e025bc6cd0189893bfc518844ac9e3b31d2 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 16 Jul 2015 17:16:24 -0700 Subject: [PATCH 072/330] Adding support for hourly s3 data ingestion from secor: Summary: - Add the capability of partition the data file in S3 using hourly folder: s3://pinlogs/secor_raw_logs/topic1/dt=2015-07-07/hr=05/ This way, the data file will be on S3 much sooner (hourly vs daily), people can check the result on S3 much sooner and this also opens the door to have an faster hourly data pipeline on Hadoop side as well. The hr folder values are from 00-23 - To trigger the hourly partition and finalization, add the following parameter in your secor config: * # partitioner.granularity.hour=true And change the upload threshold to less than one hour: * secor.max.file.age.seconds=3000 - Change the hive partition registration code to register partition using both dt and hr column (it does require the HIVE table to be created or altered to have both dt and hr as the partition columns) - The enhancements are done through the following: * Introduce a new interface Partitioner which knows what the last partition period to be finalized (generating SUCCESS file) and knows how to find the previous partition periods to be finalized upon. * Change the TimestampedMessageParser to implement Partitioner, allow it to extract both the dt and hr from the timestamp field, and knows how to traverse backwards to find the previous partitions * Change the Finalizer to work with the Partitioner to loop through the list of ready-to-be finalized partitions for both the hr and dt folder to generate the SUCCESS file and do hive registration - Added more unit test on message parser on hourly behavior - Added more E2E tests to cover the partitioner and hourly ingestion Test Plan: Added both unit tests and e2e tests, and tested manually for a new topic on S3 --- src/main/config/secor.common.properties | 16 ++ .../config/secor.dev.hr.partition.properties | 2 +- .../pinterest/secor/common/SecorConfig.java | 18 +- .../secor/parser/PartitionFinalizer.java | 155 +++++++++--------- .../pinterest/secor/parser/Partitioner.java | 7 + .../parser/TimestampedMessageParser.java | 47 ++++-- src/main/scripts/run_tests.sh | 12 +- .../secor/parser/JsonMessageParserTest.java | 101 +++++++++++- 8 files changed, 239 insertions(+), 119 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index f6c9c2f9b..1c296212d 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -86,6 +86,22 @@ secor.offsets.per.partition=10000000 # How long does it take for secor to forget a topic partition. Applies to stats generation only. secor.topic_partition.forget.seconds=600 +# 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=true + +# 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 starup Secor will clean up directories and files under secor.local.path # that are older than this value. secor.local.log.delete.age.hours=-1 diff --git a/src/main/config/secor.dev.hr.partition.properties b/src/main/config/secor.dev.hr.partition.properties index 2b190ae37..95c7017c2 100644 --- a/src/main/config/secor.dev.hr.partition.properties +++ b/src/main/config/secor.dev.hr.partition.properties @@ -6,6 +6,6 @@ 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 -message.timestamp.using.hour=true +partitioner.granularity.hour=true ostrich.port=9998 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 7d66ddd61..23621750c 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -211,16 +211,8 @@ public String getMessageTimestampInputPattern() { return getString("message.timestamp.input.pattern"); } - public boolean getMessageTimestampUsingHour() { - return getBoolean("message.timestamp.using.hour", false); - } - - public int getFinalizerLagSecond() { - return getInt("finalizer.lag.second", 3600); - } - public int getFinalizerLookbackPeriods() { - return getInt("finalizer.lookback.periods", 10); + return getInt("secor.finalizer.lookback.periods", 10); } public String getHivePrefix() { @@ -247,6 +239,10 @@ public String getZookeeperPath() { return getString("secor.zookeeper.path"); } + public boolean getBoolean(String name, boolean defaultValue) { + return mProperties.getBoolean(name, defaultValue); + } + private void checkProperty(String name) { if (!mProperties.containsKey(name)) { throw new RuntimeException("Failed to find required configuration option '" + @@ -268,10 +264,6 @@ private int getInt(String name, int defaultValue) { return mProperties.getInt(name, defaultValue); } - private boolean getBoolean(String name, boolean defaultValue) { - return mProperties.getBoolean(name, defaultValue); - } - private long getLong(String name) { return mProperties.getLong(name); } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index b9297d589..58e12fa68 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -85,98 +85,95 @@ private String[] getFinalizedUptoPartitions(String topic) throws Exception { return mMessageParser.getFinalizedUptoPartitions(lastMessages, committedMessages); } - private void finalizePartitionsUpTo(String topic, String[] partitions) throws Exception { + private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throws Exception { final String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); - // Walk through each dimension of the partitions array - // First finalize the hourly partitions, and then move on to daily partitions - for (int dim = partitions.length; dim > 0; dim--) { - String[] uptoPartitions = Arrays.copyOf(partitions, dim); - LOG.info("Finalize up to (but not include) {} for dim: {}", uptoPartitions, dim); + LOG.info("Finalize up to (but not include) {}, dim: {}", + uptoPartitions, uptoPartitions.length); - 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(s3Prefix, topic, previous, - mConfig.getGeneration(), 0, 0, mFileExtension); - String logFileDir = logFilePath.getLogFileDir(); - 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); + 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(s3Prefix, topic, previous, + mConfig.getGeneration(), 0, 0, mFileExtension); + String logFileDir = logFilePath.getLogFileDir(); + if (FileUtil.exists(logFileDir)) { + String successFilePath = logFileDir + "/_SUCCESS"; + if (FileUtil.exists(successFilePath)) { + LOG.info( + "SuccessFile exist already, short circuit return. " + successFilePath); + break; } - previous = mMessageParser.getPreviousPartitions(previous); + LOG.info("Folder {} exists and ready to be finalized.", logFileDir); + toBeFinalized.push(previous); + } else { + LOG.info("Folder {} doesn't exist, skip", logFileDir); } + previous = mMessageParser.getPreviousPartitions(previous); + } - LOG.info("To be finalized partitions: {}", toBeFinalized); - if (toBeFinalized.isEmpty()) { - LOG.warn("There is no partitions to be finalized for dim: {}", dim); - continue; - } + LOG.info("To be finalized partitions: {}", toBeFinalized); + if (toBeFinalized.isEmpty()) { + LOG.warn("There is no partitions to be finalized."); + return; + } - // 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 (dim == partitions.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 hivePrefix = null; - try { - hivePrefix = mConfig.getHivePrefix(); - } catch (RuntimeException ex) { - LOG.warn("HivePrefix is not defined. Skip hive registration"); + // 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(","); } - if (hivePrefix != null) { - mQuboleClient.addPartition(hivePrefix + topic, sb.toString()); - } - } catch (Exception e) { - LOG.error("failed to finalize topic " + topic, e); - continue; + sb.append(parts[0]); + sb.append("='"); + sb.append(parts[1]); + sb.append("'"); + } + LOG.info("Hive partition string: " + sb); + String hivePrefix = null; + try { + hivePrefix = mConfig.getHivePrefix(); + } catch (RuntimeException ex) { + LOG.warn("HivePrefix is not defined. Skip hive registration"); + } + if (hivePrefix != null) { + mQuboleClient.addPartition(hivePrefix + topic, 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(s3Prefix, topic, current, - mConfig.getGeneration(), 0, 0, mFileExtension); - String logFileDir = logFilePath.getLogFileDir(); - String successFilePath = logFileDir + "/_SUCCESS"; + // Generate the SUCCESS file at the end + LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, current, + mConfig.getGeneration(), 0, 0, mFileExtension); + String logFileDir = logFilePath.getLogFileDir(); + String successFilePath = logFileDir + "/_SUCCESS"; - LOG.info("touching file {}", successFilePath); - FileUtil.touch(successFilePath); - } + LOG.info("touching file {}", successFilePath); + FileUtil.touch(successFilePath); } + } public void finalizePartitions() throws Exception { diff --git a/src/main/java/com/pinterest/secor/parser/Partitioner.java b/src/main/java/com/pinterest/secor/parser/Partitioner.java index d7a4f78e9..6253758e0 100644 --- a/src/main/java/com/pinterest/secor/parser/Partitioner.java +++ b/src/main/java/com/pinterest/secor/parser/Partitioner.java @@ -90,6 +90,13 @@ String[] getFinalizedUptoPartitions(List lastMessages, * 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 */ diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index a13a252db..c0f330044 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -31,6 +31,9 @@ public abstract class TimestampedMessageParser extends MessageParser implements 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 SimpleDateFormat mDtFormatter = new SimpleDateFormat("yyyy-MM-dd"); private static final SimpleDateFormat mHrFormatter = new SimpleDateFormat("HH"); private static final SimpleDateFormat mDtHrFormatter = new SimpleDateFormat("yyyy-MM-dd-HH"); @@ -42,17 +45,19 @@ public abstract class TimestampedMessageParser extends MessageParser implements } private final boolean mUsingHourly; - private final long mLagInSeconds; public TimestampedMessageParser(SecorConfig config) { super(config); - mUsingHourly = config.getMessageTimestampUsingHour(); - mLagInSeconds = config.getFinalizerLagSecond(); - LOG.info("UsingHourly: {}, lagInSeconds: {} ", mUsingHourly, mLagInSeconds); + mUsingHourly = usingHourly(config); + LOG.info("UsingHourly: {}", mUsingHourly); } public abstract long extractTimestampMillis(final Message message) throws Exception; + static boolean usingHourly(SecorConfig config) { + return config.getBoolean("partitioner.granularity.hour", false); + } + protected static long toMillis(final long timestamp) { final long nanosecondDivider = (long) Math.pow(10, 9 + 9); final long millisecondDivider = (long) Math.pow(10, 9 + 3); @@ -131,23 +136,37 @@ public String[] getFinalizedUptoPartitions(List lastMessages, } // add the safety lag for late-arrival messages - long lag = mLagInSeconds * 1000L; - LOG.info("Originally: {}, adjust down {}", minMillis, lag); - return generatePartitions(minMillis - lag, mUsingHourly); + minMillis -= 3600L * 1000L; + LOG.info("adjusted millis {}", minMillis); + return generatePartitions(minMillis, mUsingHourly); } @Override public String[] getPreviousPartitions(String[] partitions) throws Exception { long millis = parsePartitions(partitions); - long delta; - if (partitions.length == 1) { - delta = 3600L * 24 * 1000L; - } else if (partitions.length == 2) { - delta = 3600L * 1000L; + boolean usingHourly = mUsingHourly; + if (mUsingHourly && millis % DAY_IN_MILLIS == 0) { + // On the day boundary, if the currrent 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 { - throw new RuntimeException("Unsupported partitions: " + partitions.length); + long delta = mUsingHourly ? HOUR_IN_MILLIS : DAY_IN_MILLIS; + millis -= delta; } - return generatePartitions(millis - delta, partitions.length == 2); + return generatePartitions(millis, usingHourly); } } diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 2944b4724..d42f7a8dd 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -54,7 +54,7 @@ READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriterFacto # 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=${SECOR_WAIT_TIME:-30} +WAIT_TIME=${SECOR_WAIT_TIME:-120} BASE_DIR=$(dirname $0) CONF_DIR=${BASE_DIR}/.. @@ -192,6 +192,10 @@ verify() { 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 @@ -288,7 +292,7 @@ post_and_finalizer_verify_test() { OLD_ADDITIONAL_OPTS=${ADDITIONAL_OPTS} if [ $1 = "hr" ]; then - ADDITIONAL_OPTS="${ADDITIONAL_OPTS} -Dmessage.timestamp.using.hour=true -Dfinalizer.lookback.periods=30" + 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 @@ -357,8 +361,8 @@ move_offset_back_test() { set_offsets_in_zookeeper 2 post_messages $((${MESSAGES}*9/10)) 0 - echo "Waiting $((${WAIT_TIME}*3)) sec for Secor to upload logs to s3" - sleep $((${WAIT_TIME}*3)) + echo "Waiting $((${WAIT_TIME}*2)) sec for Secor to upload logs to s3" + sleep $((${WAIT_TIME}*2)) # 4 because we skipped 2 messages per topic partition and there are 2 partitions per topic. verify $((${MESSAGES}-4)) 0 diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index f8157f9f4..96c0cc687 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -18,12 +18,17 @@ import com.pinterest.secor.common.*; import com.pinterest.secor.message.Message; + +import java.util.Arrays; +import java.util.List; 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.ArrayList; + @RunWith(PowerMockRunner.class) public class JsonMessageParserTest extends TestCase { @@ -39,15 +44,15 @@ public void setUp() throws Exception { Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); 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"); + "{\"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, messageWithSecondsTimestamp); 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"); + "{\"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, messageWithMillisTimestamp); 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"); + "{\"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, messageWithMillisFloatTimestamp); byte messageWithoutTimestamp[] = @@ -59,9 +64,9 @@ public void setUp() throws Exception { 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.extractTimestampMillis(mMessageWithSecondsTimestamp)); + assertEquals(1405911096123l, jsonMessageParser.extractTimestampMillis(mMessageWithMillisTimestamp)); + assertEquals(1405911096123l, jsonMessageParser.extractTimestampMillis(mMessageWithMillisFloatTimestamp)); // Return 0 if there's no timestamp, for any reason. @@ -101,11 +106,11 @@ public void testExtractPartitions() throws Exception { @Test public void testExtractHourlyPartitions() throws Exception { - Mockito.when(mConfig.getMessageTimestampUsingHour()).thenReturn(true); + Mockito.when(TimestampedMessageParser.usingHourly(mConfig)).thenReturn(true); JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); String expectedDtPartition = "dt=2014-07-21"; - String expectedHrPartition = "hr=19"; + String expectedHrPartition = "hr=02"; String resultSeconds[] = jsonMessageParser.extractPartitions(mMessageWithSecondsTimestamp); assertEquals(2, resultSeconds.length); @@ -118,4 +123,84 @@ public void testExtractHourlyPartitions() throws Exception { 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); + } + } + } From 1ac56a94274e255ff2a3591b5e722514f2f6eca1 Mon Sep 17 00:00:00 2001 From: lwoessner Date: Wed, 22 Jul 2015 09:12:42 -0600 Subject: [PATCH 073/330] take out extra line breaks for style --- .../com/pinterest/secor/message/Message.java | 41 ++++++++----------- .../pinterest/secor/message/MessageTest.java | 12 +++--- 2 files changed, 21 insertions(+), 32 deletions(-) diff --git a/src/main/java/com/pinterest/secor/message/Message.java b/src/main/java/com/pinterest/secor/message/Message.java index d348878c8..f1349f3b9 100644 --- a/src/main/java/com/pinterest/secor/message/Message.java +++ b/src/main/java/com/pinterest/secor/message/Message.java @@ -20,7 +20,6 @@ import java.io.OutputStream; import java.lang.String; - /** * Message represents a raw Kafka log message. * @@ -28,67 +27,59 @@ */ public class Message { - private String mTopic; private int mKafkaPartition; private long mOffset; private byte[] mPayload; - protected String fieldsToString() { - return "topic='" + mTopic + '\'' + ", kafkaPartition=" + mKafkaPartition + ", offset=" + mOffset + ", payload=" - + new String(mPayload); + return "topic='" + mTopic + '\'' + ", kafkaPartition=" + + mKafkaPartition + ", offset=" + mOffset + ", payload=" + + new String(mPayload); } - @Override public String toString() { - return "Message{" + fieldsToString() + '}'; + return "Message{" + fieldsToString() + '}'; } - public Message(String topic, int kafkaPartition, long offset, byte[] payload) { - mTopic = topic; - mKafkaPartition = kafkaPartition; - mOffset = offset; - mPayload = payload; + mTopic = topic; + mKafkaPartition = kafkaPartition; + mOffset = offset; + mPayload = payload; - if (mPayload == null) { - mPayload = new byte[0]; - } + if (mPayload == null) { + mPayload = new byte[0]; + } } - public String getTopic() { - return mTopic; + return mTopic; } - public int getKafkaPartition() { - return mKafkaPartition; + return mKafkaPartition; } - public long getOffset() { - return mOffset; + return mOffset; } - public byte[] getPayload() { - return mPayload; + return mPayload; } - public void write(OutputStream output) throws IOException { - output.write(mPayload); + output.write(mPayload); } } diff --git a/src/test/java/com/pinterest/secor/message/MessageTest.java b/src/test/java/com/pinterest/secor/message/MessageTest.java index e49d894e4..e7a4ed1ea 100644 --- a/src/test/java/com/pinterest/secor/message/MessageTest.java +++ b/src/test/java/com/pinterest/secor/message/MessageTest.java @@ -4,17 +4,15 @@ import org.junit.Test; - - public class MessageTest { - @Test public void testNullPayload() { - Message message = new Message("testTopic", 0, 123, null); - System.out.println(message); - - // no assert necessary, just making sure it does not throw a NullPointerException + Message message = new Message("testTopic", 0, 123, null); + System.out.println(message); + + // no assert necessary, just making sure it does not throw a + // NullPointerException } } From 9b8afe08ebefd466ef66175a6f9e90110cebf03d Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Fri, 24 Jul 2015 18:58:29 -0700 Subject: [PATCH 074/330] Add support to enable deploy/publish through SonaType to maven central Added all the required change to pom to allow build publishing to maven central through SonaType.org. Most of the changes are following examples in: http://central.sonatype.org/pages/ossrh-guide.html The first version is published to maven under (including both sources and javadoc jars): https://repo1.maven.org/maven2/com/pinterest/secor/0.1 Since ver 0.1 is published, our current SNAPSHOT build will be upgraded to 0.2-SNAPSHOT To publish the snapshot build (you need to have SonaType account and GnuPGP signing tools configured): # make sure the version in pom.xml ends with SNAPSHOT mvn clean deploy # the artifacts will stay in Sonatype repo (not going into maven central) To publish a release build # make sure the version in pom.xml not ends with SNAPSHOT mvn clean deploy -P release # the artifacts are pushed all the way through to maven central # when the release build is published to maven central, make sure the version in pom.xml is upgraded to next-version-SNAPSHOT to reflect we are not building snapshot for next version until it's ready to be version released. --- pom.xml | 103 ++++++++++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 101 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 963f7a091..5ce2e7b41 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.2-SNAPSHOT jar + secor Kafka to s3 logs exporter + https://github.com/pinterest/secor @@ -19,6 +21,31 @@ + + + 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 @@ -26,6 +53,17 @@ UTF-8 + + + ossrh + https://oss.sonatype.org/service/local/staging/deploy/maven2/ + + + ossrh + https://oss.sonatype.org/content/repositories/snapshots + + + Twitter public Maven repo @@ -326,6 +364,67 @@ + + org.sonatype.plugins + nexus-staging-maven-plugin + 1.6.3 + true + + ossrh + https://oss.sonatype.org/ + true + + + + + + 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 + + + + + + + + From ed1e0993cda523cdb0c59b0688edd621f0c890af Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Fri, 17 Jul 2015 15:56:31 -0500 Subject: [PATCH 075/330] Configurable prefix for metric names. --- src/main/config/secor.common.properties | 3 +++ .../com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../pinterest/secor/tools/ProgressMonitor.java | 16 ++++++++++++++-- 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index f6c9c2f9b..6b7be938d 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -107,6 +107,9 @@ tsdb.hostport= # Regex of topics that are not exported to TSDB. monitoring.blacklist.topics= +# Prefix of exported statss. +monitoring.prefix=secor + # 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= diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 2e03856f2..8649c16d9 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -203,6 +203,10 @@ public String getMonitoringBlacklistTopics() { return getString("monitoring.blacklist.topics"); } + public String getMonitoringPrefix() { + return getString("monitoring.prefix"); + } + public String getMessageTimestampName() { return getString("message.timestamp.name"); } diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 032ad482a..887f0cc2f 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -16,6 +16,8 @@ */ 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; @@ -60,6 +62,7 @@ public class ProgressMonitor { private ZookeeperConnector mZookeeperConnector; private KafkaClient mKafkaClient; private MessageParser mMessageParser; + private String mPrefix; public ProgressMonitor(SecorConfig config) throws Exception @@ -69,6 +72,11 @@ public ProgressMonitor(SecorConfig config) mKafkaClient = new KafkaClient(mConfig); mMessageParser = (MessageParser) ReflectionUtil.createMessageParser( mConfig.getMessageParserClass(), mConfig); + + mPrefix = mConfig.getMonitoringPrefix(); + if (Strings.isNullOrEmpty(mPrefix)) { + mPrefix = "secor"; + } } private void makeRequest(String body) throws IOException { @@ -199,8 +207,8 @@ private List getStats() throws Exception { ); long timestamp = System.currentTimeMillis() / 1000; - stats.add(Stat.createInstance("secor.lag.offsets", tags, Long.toString(offsetLag), timestamp)); - stats.add(Stat.createInstance("secor.lag.seconds", tags, Long.toString(timestampMillisLag / 1000), timestamp)); + 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, @@ -212,6 +220,10 @@ private List getStats() throws Exception { 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).extractTimestampMillis(message); From dcec926bb20007b7b20fba3196ca6825129a5ed3 Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Wed, 1 Jul 2015 11:19:18 -0500 Subject: [PATCH 076/330] Support uploads using the AWS SDK TransferManager. This provides more flexibility for credentials and region choice. The core upload functionality has been moved out of `Uploader` and into `UploadManager` implementations. The manager to use can be set through the configuration file. --- pom.xml | 5 ++ src/main/config/secor.common.properties | 16 ++++ .../pinterest/secor/common/LogFilePath.java | 4 + .../pinterest/secor/common/SecorConfig.java | 12 +++ .../pinterest/secor/consumer/Consumer.java | 5 +- .../secor/uploader/FutureHandle.java | 36 ++++++++ .../secor/uploader/HadoopS3UploadManager.java | 66 ++++++++++++++ .../com/pinterest/secor/uploader/Handle.java | 26 ++++++ .../secor/uploader/S3UploadHandle.java | 38 +++++++++ .../secor/uploader/S3UploadManager.java | 85 +++++++++++++++++++ .../secor/uploader/UploadManager.java | 34 ++++++++ .../pinterest/secor/uploader/Uploader.java | 45 +++------- .../pinterest/secor/util/ReflectionUtil.java | 27 +++++- .../secor/uploader/UploaderTest.java | 11 ++- 14 files changed, 371 insertions(+), 39 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/uploader/FutureHandle.java create mode 100644 src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java create mode 100644 src/main/java/com/pinterest/secor/uploader/Handle.java create mode 100644 src/main/java/com/pinterest/secor/uploader/S3UploadHandle.java create mode 100644 src/main/java/com/pinterest/secor/uploader/S3UploadManager.java create mode 100644 src/main/java/com/pinterest/secor/uploader/UploadManager.java diff --git a/pom.xml b/pom.xml index df0802ab1..153f25a91 100644 --- a/pom.xml +++ b/pom.xml @@ -91,6 +91,11 @@ protobuf-java 2.6.1 + + com.amazonaws + aws-java-sdk-s3 + 1.10.2 + net.java.dev.jets3t jets3t diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index f4701d56a..d3a41dfc2 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -28,6 +28,18 @@ aws.secret.key= # 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= + # Zookeeper config. zookeeper.session.timeout.ms=3000 zookeeper.sync.time.ms=200 @@ -149,3 +161,7 @@ secor.file.reader.writer.factory=com.pinterest.secor.io.impl.SequenceFileReaderW # 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 diff --git a/src/main/java/com/pinterest/secor/common/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java index 753288c0d..f73f8b6a9 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -114,6 +114,10 @@ public LogFilePath(String prefix, String path) { mOffset = Long.parseLong(basenameElements[2]); } + public LogFilePath withPrefix(String prefix) { + return new LogFilePath(prefix, mTopic, mPartitions, mGeneration, mKafkaPartition, mOffset, mExtension); + } + public String getLogFileParentDir() { ArrayList elements = new ArrayList(); elements.add(mPrefix); diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 64bc91f35..e833d413d 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -167,6 +167,10 @@ public String getMessageParserClass() { return getString("secor.message.parser.class"); } + public String getUploadManagerClass() { + return getString("secor.upload.manager.class"); + } + public int getTopicPartitionForgetSeconds() { return getInt("secor.topic_partition.forget.seconds"); } @@ -187,6 +191,14 @@ public String getAwsSecretKey() { return getString("aws.secret.key"); } + public String getAwsEndpoint() { + return getString("aws.endpoint"); + } + + public String getAwsRegion() { + return getString("aws.region"); + } + public String getQuboleApiToken() { return getString("qubole.api.token"); } diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 969819488..34155fa25 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -23,6 +23,7 @@ import com.pinterest.secor.message.ParsedMessage; import com.pinterest.secor.parser.MessageParser; import com.pinterest.secor.uploader.Uploader; +import com.pinterest.secor.uploader.UploadManager; import com.pinterest.secor.reader.MessageReader; import com.pinterest.secor.util.ReflectionUtil; import com.pinterest.secor.writer.MessageWriter; @@ -66,9 +67,11 @@ private void init() throws Exception { mOffsetTracker = new OffsetTracker(); mMessageReader = new MessageReader(mConfig, mOffsetTracker); FileRegistry fileRegistry = new FileRegistry(mConfig); + UploadManager uploadManager = ReflectionUtil.createUploadManager(mConfig.getUploadManagerClass(), mConfig); + + mUploader = new Uploader(mConfig, mOffsetTracker, fileRegistry, uploadManager); mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, fileRegistry); mMessageParser = ReflectionUtil.createMessageParser(mConfig.getMessageParserClass(), mConfig); - mUploader = new Uploader(mConfig, mOffsetTracker, fileRegistry); mUnparsableMessages = 0.; } 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/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java new file mode 100644 index 000000000..6e9065cf9 --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -0,0 +1,66 @@ +/** + * 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); + + private static final ExecutorService executor = Executors.newFixedThreadPool(256); + + public HadoopS3UploadManager(SecorConfig config) { + super(config); + } + + public Handle upload(LogFilePath localPath) throws Exception { + String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); + LogFilePath s3Path = localPath.withPrefix(s3Prefix); + final String localLogFilename = localPath.getLogFilePath(); + final String s3LogFilename = s3Path.getLogFilePath(); + LOG.info("uploading file {} to {}", localLogFilename, s3LogFilename); + + final Future f = executor.submit(new Runnable() { + @Override + public void run() { + try { + FileUtil.moveToS3(localLogFilename, s3LogFilename); + } 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..ab5a8dbb1 --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -0,0 +1,85 @@ +/** + * 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.amazonaws.auth.AWSCredentials; +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.transfer.Upload; +import com.amazonaws.services.s3.transfer.TransferManager; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; + +/** + * Manages uploads to S3 using the TransferManager class from the AWS + * SDK. + * + * It will use the aws.access.key and aws.secret.key configuration + * settings if they are non-empty; otherwise, it will use the SDK's + * default credential provider chain (supports environment variables, + * system properties, credientials file, and IAM credentials). + * + * @author Liam Stewart (liam.stewart@gmail.com) + */ +public class S3UploadManager extends UploadManager { + private static final Logger LOG = LoggerFactory.getLogger(S3UploadManager.class); + + private TransferManager mManager; + + public S3UploadManager(SecorConfig config) { + super(config); + + String accessKey = mConfig.getAwsAccessKey(); + String secretKey = mConfig.getAwsSecretKey(); + String endpoint = mConfig.getAwsEndpoint(); + String region = mConfig.getAwsRegion(); + AmazonS3 client; + + if (accessKey.isEmpty() || secretKey.isEmpty()) { + client = new AmazonS3Client(); + } else { + client = new AmazonS3Client(new BasicAWSCredentials(accessKey, secretKey)); + } + + 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 s3Key = localPath.withPrefix(mConfig.getS3Path()).getLogFilePath(); + File localFile = new File(localPath.getLogFilePath()); + + LOG.info("uploading file {} to s3://{}/{}", localFile, s3Bucket, s3Key); + + Upload upload = mManager.upload(s3Bucket, s3Key, localFile); + return new S3UploadHandle(upload); + } +} 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 65cb1e617..ef9c8e3e2 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -33,9 +33,6 @@ import java.io.IOException; import java.util.*; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; /** * Uploader applies a set of policies to determine if any of the locally stored files should be @@ -46,49 +43,29 @@ public class Uploader { private static final Logger LOG = LoggerFactory.getLogger(Uploader.class); - private static final ExecutorService executor = Executors.newFixedThreadPool(256); - private SecorConfig mConfig; private OffsetTracker mOffsetTracker; private FileRegistry mFileRegistry; private ZookeeperConnector mZookeeperConnector; + private UploadManager mUploadManager; - public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry) { - this(config, offsetTracker, fileRegistry, new ZookeeperConnector(config)); + public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, + UploadManager uploadManager) { + this(config, offsetTracker, fileRegistry, uploadManager, + new ZookeeperConnector(config)); } // For testing use only. public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, + UploadManager uploadManager, ZookeeperConnector zookeeperConnector) { mConfig = config; mOffsetTracker = offsetTracker; mFileRegistry = fileRegistry; + mUploadManager = uploadManager; mZookeeperConnector = zookeeperConnector; } - private Future 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()); - final String localLogFilename = localPath.getLogFilePath(); - final String s3LogFilename = s3Path.getLogFilePath(); - LOG.info("uploading file {} to {}", localLogFilename, s3LogFilename); - return executor.submit(new Runnable() { - @Override - public void run() { - try { - FileUtil.moveToS3(localLogFilename, s3LogFilename); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - }); - } - private void uploadFiles(TopicPartition topicPartition) throws Exception { long committedOffsetCount = mOffsetTracker.getTrueCommittedOffsetCount(topicPartition); long lastSeenOffset = mOffsetTracker.getLastSeenOffset(topicPartition); @@ -112,12 +89,12 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { // Deleting writers closes their streams flushing all pending data to the disk. mFileRegistry.deleteWriters(topicPartition); Collection paths = mFileRegistry.getPaths(topicPartition); - List> uploadFutures = new ArrayList>(); + List> uploadHandles = new ArrayList>(); for (LogFilePath path : paths) { - uploadFutures.add(upload(path)); + uploadHandles.add(mUploadManager.upload(path)); } - for (Future uploadFuture : uploadFutures) { - uploadFuture.get(); + for (Handle uploadHandle : uploadHandles) { + uploadHandle.get(); } mFileRegistry.deleteTopicPartition(topicPartition); mZookeeperConnector.setCommittedOffsetCount(topicPartition, lastSeenOffset + 1); diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 30420f8fe..fb71cc247 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -23,6 +23,7 @@ import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.io.FileReaderWriterFactory; import com.pinterest.secor.parser.MessageParser; +import com.pinterest.secor.uploader.UploadManager; import org.apache.hadoop.io.compress.CompressionCodec; /** @@ -33,6 +34,30 @@ * @author Silas Davis (github-code@silasdavis.net) */ public class ReflectionUtil { + /** + * 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 a MessageParser from it's fully qualified class name. @@ -109,4 +134,4 @@ public static FileReader createFileReader(String className, LogFilePath logFileP throws Exception { return createFileReaderWriterFactory(className).BuildFileReader(logFilePath, codec); } -} \ No newline at end of file +} diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index a6a90e1a1..48468e71d 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -49,8 +49,10 @@ private static class TestUploader extends Uploader { private FileReader mReader; public TestUploader(SecorConfig config, OffsetTracker offsetTracker, - FileRegistry fileRegistry, ZookeeperConnector zookeeperConnector) { - super(config, offsetTracker, fileRegistry, zookeeperConnector); + FileRegistry fileRegistry, + UploadManager uploadManager, + ZookeeperConnector zookeeperConnector) { + super(config, offsetTracker, fileRegistry, uploadManager, zookeeperConnector); mReader = Mockito.mock(FileReader.class); } @@ -73,6 +75,7 @@ public FileReader getReader() { private OffsetTracker mOffsetTracker; private FileRegistry mFileRegistry; private ZookeeperConnector mZookeeperConnector; + private UploadManager mUploadManager; private TestUploader mUploader; @@ -99,8 +102,10 @@ public void setUp() throws Exception { Mockito.when(mFileRegistry.getTopicPartitions()).thenReturn( topicPartitions); + mUploadManager = new HadoopS3UploadManager(mConfig); + mZookeeperConnector = Mockito.mock(ZookeeperConnector.class); - mUploader = new TestUploader(mConfig, mOffsetTracker, mFileRegistry, + mUploader = new TestUploader(mConfig, mOffsetTracker, mFileRegistry, mUploadManager, mZookeeperConnector); } From d5a60012e1b24250b3288c7bae194e6417e1bf04 Mon Sep 17 00:00:00 2001 From: lwoessner Date: Tue, 11 Aug 2015 11:12:24 -0600 Subject: [PATCH 077/330] force build --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 963f7a091..196abe799 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" 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 secor 0.1-SNAPSHOT From f835c3d368b6454a26217de901609b2671d291dd Mon Sep 17 00:00:00 2001 From: Pawel Garbacki Date: Sun, 23 Aug 2015 12:51:56 -0700 Subject: [PATCH 078/330] Update README.md --- README.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/README.md b/README.md index 5d579a469..aca03246b 100644 --- a/README.md +++ b/README.md @@ -120,6 +120,14 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Zack Dever](https://github.com/zackdever) * [Leo Woessner](https://github.com/estezz) +## 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) + ## Help If you have any questions or comments, you can reach us at [secor-users@googlegroups.com](https://groups.google.com/forum/#!forum/secor-users) From b09e2255e999a5881c19052f38ff39bae29b490b Mon Sep 17 00:00:00 2001 From: Pawel Garbacki Date: Sun, 23 Aug 2015 12:52:28 -0700 Subject: [PATCH 079/330] Update README.md --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index aca03246b..6e7b93307 100644 --- a/README.md +++ b/README.md @@ -123,10 +123,10 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l ## 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) + * [Pinterest](https://www.pinterest.com) + * [Strava](https://www.strava.com) + * [TiVo](https://www.tivo.com) + * [Yelp](http://www.yelp.com) ## Help From 0ad78d7f6d17b994e754b70c2d8aae42a6dd828f Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Tue, 25 Aug 2015 16:39:52 -0500 Subject: [PATCH 080/330] Close consumers used in get(Committed|Last)Message. This stops connections from being opened and then immediately orphaned on calls to these methods. --- .../pinterest/secor/common/KafkaClient.java | 57 ++++++++++++------- 1 file changed, 38 insertions(+), 19 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index ab33ef149..421ca3f90 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -60,9 +60,10 @@ private HostAndPort findLeader(TopicPartition topicPartition) { SimpleConsumer consumer = null; try { LOG.info("looking up leader for topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); - consumer = new SimpleConsumer(mConfig.getKafkaSeedBrokerHost(), - mConfig.getKafkaSeedBrokerPort(), - 100000, 64 * 1024, "leaderLookup"); + consumer = createConsumer( + mConfig.getKafkaSeedBrokerHost(), + mConfig.getKafkaSeedBrokerPort(), + "leaderLookup"); List topics = new ArrayList(); topics.add(topicPartition.getTopic()); TopicMetadataRequest request = new TopicMetadataRequest(topics); @@ -135,20 +136,24 @@ private Message getMessage(TopicPartition topicPartition, long offset, messageAndOffset.offset(), payloadBytes); } - public SimpleConsumer createConsumer(TopicPartition topicPartition) { + private SimpleConsumer createConsumer(String host, int port, String clientName) { + return new SimpleConsumer(host, port, 100000, 64 * 1024, clientName); + } + + public SimpleConsumer createConsumer(TopicPartition topicPartition) { HostAndPort leader = findLeader(topicPartition); LOG.info("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); @@ -167,20 +172,34 @@ 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); + } finally { + if (consumer != null) { + consumer.close(); + } } - SimpleConsumer consumer = createConsumer(topicPartition); - return getMessage(topicPartition, committedOffset, consumer); } } From e00fac6be98d8dda0739f591ab429a8584d85b78 Mon Sep 17 00:00:00 2001 From: Space Date: Wed, 22 Apr 2015 19:01:37 -0700 Subject: [PATCH 081/330] Support s3a hadoop file system. Added tests to run through both with s3n and s3a Updated hadoop-aws and jets3t packages. --- Makefile | 1 + pom.xml | 23 +++++++-- src/main/config/log4j.dev.properties | 1 + src/main/config/log4j.prod.properties | 1 + src/main/config/secor.common.properties | 5 ++ .../pinterest/secor/common/SecorConfig.java | 5 ++ .../secor/parser/PartitionFinalizer.java | 6 +-- .../secor/tools/LogFileVerifier.java | 8 +-- .../secor/uploader/HadoopS3UploadManager.java | 3 +- .../com/pinterest/secor/util/FileUtil.java | 27 ++++++---- src/main/scripts/run_tests.sh | 49 +++++++++++-------- src/test/config/core-site.xml | 7 +++ src/test/config/jets3t.properties | 1 + .../secor/uploader/UploaderTest.java | 5 +- 14 files changed, 93 insertions(+), 49 deletions(-) create mode 100644 src/test/config/core-site.xml diff --git a/Makefile b/Makefile index b85de0ce7..90cda6b02 100644 --- a/Makefile +++ b/Makefile @@ -15,6 +15,7 @@ integration: build @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 diff --git a/pom.xml b/pom.xml index 153f25a91..c281fe8eb 100644 --- a/pom.xml +++ b/pom.xml @@ -99,7 +99,7 @@ net.java.dev.jets3t jets3t - 0.7.1 + 0.9.4 org.apache.kafka @@ -153,13 +153,30 @@ org.apache.hadoop - hadoop-core - 1.2.1 + hadoop-common + 2.7.0 + + + org.apache.hadoop + 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 + 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.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 d3a41dfc2..5731f1b33 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -21,6 +21,7 @@ secor.kafka.topic_filter=.* # AWS authentication credentials. +# Leave empty if using IAM role-based authentication with s3a filesystem. aws.access.key= aws.secret.key= @@ -40,6 +41,10 @@ aws.secret.key= aws.region= aws.endpoint= +# Hadoop filesystem to use. Choices are s3n or s3a. +# See https://wiki.apache.org/hadoop/AmazonS3 for details. +secor.s3.filesystem=s3n + # Zookeeper config. zookeeper.session.timeout.ms=3000 zookeeper.sync.time.ms=200 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index e833d413d..3a9ec7976 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -135,6 +135,8 @@ public int getMessagesPerSecond() { return getInt("secor.messages.per.second"); } + public String getS3FileSystem() { return getString("secor.s3.filesystem"); } + public String getS3Bucket() { return getString("secor.s3.bucket"); } @@ -143,6 +145,9 @@ public String getS3Path() { return getString("secor.s3.path"); } + public String getS3Prefix() { + return getS3FileSystem() + "://" + getS3Bucket() + "/" + getS3Path(); + } public String getLocalPath() { return getString("secor.local.path"); } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 58e12fa68..204034bae 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -86,8 +86,6 @@ private String[] getFinalizedUptoPartitions(String topic) throws Exception { } private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throws Exception { - final String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); - LOG.info("Finalize up to (but not include) {}, dim: {}", uptoPartitions, uptoPartitions.length); @@ -98,7 +96,7 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw // 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(s3Prefix, topic, previous, + LogFilePath logFilePath = new LogFilePath(mConfig.getS3Prefix(), topic, previous, mConfig.getGeneration(), 0, 0, mFileExtension); String logFileDir = logFilePath.getLogFileDir(); if (FileUtil.exists(logFileDir)) { @@ -165,7 +163,7 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw } // Generate the SUCCESS file at the end - LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, current, + LogFilePath logFilePath = new LogFilePath(mConfig.getS3Prefix(), topic, current, mConfig.getGeneration(), 0, 0, mFileExtension); String logFileDir = logFilePath.getLogFileDir(); String successFilePath = logFileDir + "/_SUCCESS"; diff --git a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java index 531f29cbc..348eda740 100644 --- a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java +++ b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java @@ -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; + return mConfig.getS3Prefix() + "/" + mTopic; } private void populateTopicPartitionToOffsetToFiles() throws IOException { - String prefix = getPrefix(); + String prefix = mConfig.getS3Prefix(); String topicPrefix = getTopicPrefix(); String[] paths = FileUtil.listRecursively(topicPrefix); for (String path : paths) { diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java index 6e9065cf9..22989d6e1 100644 --- a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -44,8 +44,7 @@ public HadoopS3UploadManager(SecorConfig config) { } public Handle upload(LogFilePath localPath) throws Exception { - String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); - LogFilePath s3Path = localPath.withPrefix(s3Prefix); + LogFilePath s3Path = localPath.withPrefix(mConfig.getS3Prefix()); final String localLogFilename = localPath.getLogFilePath(); final String s3LogFilename = s3Path.getLogFilePath(); LOG.info("uploading file {} to {}", localLogFilename, s3LogFilename); diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index 22ba33ed3..0e5784b31 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -18,6 +18,7 @@ import com.pinterest.secor.common.SecorConfig; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -33,19 +34,25 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class FileUtil { - private static SecorConfig mConfig = null; + private static Configuration mConf = new Configuration(true); public static void configure(SecorConfig config) { - mConfig = config; + if (config != null) { + 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), conf); + return FileSystem.get(URI.create(path), mConf); } public static String[] list(String path) throws IOException { @@ -56,7 +63,7 @@ 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://")) { paths.add(statusPath.toUri().toString()); } else { paths.add(statusPath.toUri().getPath()); @@ -119,7 +126,7 @@ 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://")) { stringPath = statusPath.toUri().toString(); } else { stringPath = statusPath.toUri().getPath(); diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index d42f7a8dd..89562f9f5 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 # @@ -52,6 +53,9 @@ 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 filesytem +S3_FILESYSTEMS=${S3_FILESYSTEMS:-s3a s3n} + # 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=${SECOR_WAIT_TIME:-120} @@ -400,26 +404,29 @@ check_for_native_libs stop_s3 start_s3 -for key in ${!READER_WRITERS[@]}; do - MESSAGE_TYPE=${key} - ADDITIONAL_OPTS=-Dsecor.file.reader.writer.factory=${READER_WRITERS[${key}]} - echo "********************************************************" - echo "Running tests for Message Type: ${MESSAGE_TYPE} and ReaderWriter: ${READER_WRITERS[${key}]}" - 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 +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 stop_s3 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 index ecd21a143..18023ea5f 100644 --- a/src/test/config/jets3t.properties +++ b/src/test/config/jets3t.properties @@ -1,3 +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/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index 48468e71d..f4fdc3f4e 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -124,8 +124,7 @@ public void testUploadFiles() throws Exception { Mockito.when( mOffsetTracker.getTrueCommittedOffsetCount(mTopicPartition)) .thenReturn(11L); - Mockito.when(mConfig.getS3Bucket()).thenReturn("some_bucket"); - Mockito.when(mConfig.getS3Path()).thenReturn("some_s3_parent_dir"); + Mockito.when(mConfig.getS3Prefix()).thenReturn("s3a://some_bucket/some_s3_parent_dir"); HashSet logFilePaths = new HashSet(); logFilePaths.add(mLogFilePath); @@ -142,7 +141,7 @@ public void testUploadFiles() throws Exception { 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/" + "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( From e42f9243ea4c38981c08fb46748d5b4bfea05e9f Mon Sep 17 00:00:00 2001 From: Pawel Garbacki Date: Wed, 26 Aug 2015 08:25:19 -0700 Subject: [PATCH 082/330] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 6e7b93307..91d5cf339 100644 --- a/README.md +++ b/README.md @@ -110,6 +110,7 @@ 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) From 03ebeb0041a973a56e7ac24bb30209b5b502f46e Mon Sep 17 00:00:00 2001 From: lwoessner Date: Wed, 26 Aug 2015 16:56:56 -0600 Subject: [PATCH 083/330] run again --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7afa1b52c..21a16b40a 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ jar secor Kafka to s3 logs exporter - https://github.com/pinterest/secor + https://github.com/pinterest/secor From 3adb064f71b240f7fbcd4ac53e60103319daeba7 Mon Sep 17 00:00:00 2001 From: Leo Woessner Date: Sat, 29 Aug 2015 09:17:29 -0600 Subject: [PATCH 084/330] remove extra spaces --- src/main/java/com/pinterest/secor/message/Message.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/main/java/com/pinterest/secor/message/Message.java b/src/main/java/com/pinterest/secor/message/Message.java index f1349f3b9..8673dc238 100644 --- a/src/main/java/com/pinterest/secor/message/Message.java +++ b/src/main/java/com/pinterest/secor/message/Message.java @@ -33,53 +33,43 @@ public class Message { private byte[] mPayload; protected String fieldsToString() { - return "topic='" + mTopic + '\'' + ", kafkaPartition=" + mKafkaPartition + ", offset=" + mOffset + ", payload=" + new String(mPayload); - } @Override public String toString() { - return "Message{" + fieldsToString() + '}'; } public Message(String topic, int kafkaPartition, long offset, byte[] payload) { - mTopic = topic; mKafkaPartition = kafkaPartition; mOffset = offset; mPayload = payload; - if (mPayload == null) { mPayload = new byte[0]; } } public String getTopic() { - return mTopic; } public int getKafkaPartition() { - return mKafkaPartition; } public long getOffset() { - return mOffset; } public byte[] getPayload() { - return mPayload; } public void write(OutputStream output) throws IOException { - output.write(mPayload); } } From 54c234a1f3dd40c249b55cea3d9bd257468e7fee Mon Sep 17 00:00:00 2001 From: Jerome Gagnon Date: Fri, 28 Aug 2015 11:10:12 -0400 Subject: [PATCH 085/330] Google cloud storage implementation Lookup for default application credential with configuration fallback Added documentation for properties file + PR Fixes --- README.md | 1 + pom.xml | 13 ++- src/main/config/secor.dev.gs.properties | 22 ++++ .../pinterest/secor/common/SecorConfig.java | 12 ++ .../secor/uploader/GsUploadManager.java | 108 ++++++++++++++++++ 5 files changed, 155 insertions(+), 1 deletion(-) create mode 100644 src/main/config/secor.dev.gs.properties create mode 100644 src/main/java/com/pinterest/secor/uploader/GsUploadManager.java diff --git a/README.md b/README.md index 91d5cf339..ec96b3c62 100644 --- a/README.md +++ b/README.md @@ -120,6 +120,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Praveen Murugesan](https://github.com/lefthandmagic) * [Zack Dever](https://github.com/zackdever) * [Leo Woessner](https://github.com/estezz) + * [Jerome Gagnon](https://github.com/jgagnon1) ## Companies who use Secor diff --git a/pom.xml b/pom.xml index 153f25a91..57392f6bc 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ 0.2-SNAPSHOT jar secor - Kafka to s3 logs exporter + Kafka to s3/gs logs exporter https://github.com/pinterest/secor @@ -220,6 +220,17 @@ java-statsd-client 3.0.2 + + com.google.apis + google-api-services-storage + v1-rev40-1.20.0 + + + com.google.guava + guava-jdk5 + + + diff --git a/src/main/config/secor.dev.gs.properties b/src/main/config/secor.dev.gs.properties new file mode 100644 index 000000000..5e0694772 --- /dev/null +++ b/src/main/config/secor.dev.gs.properties @@ -0,0 +1,22 @@ +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 + +################ +# END MUST SET # +################ + +# Google cloud storage path where files are stored within the bucket. +secor.gs.path=data + +# Application credentials configuration file +# https://developers.google.com/identity/protocols/application-default-credentials +secor.gs.credentials.path=google_app_credentials.json \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index e833d413d..7b992f820 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -255,6 +255,18 @@ 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 boolean getBoolean(String name, boolean defaultValue) { return mProperties.getBoolean(name, defaultValue); } 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..9d0a54bea --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java @@ -0,0 +1,108 @@ +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.http.FileContent; +import com.google.api.client.http.HttpTransport; +import com.google.api.client.json.JsonFactory; +import com.google.api.client.json.jackson2.JacksonFactory; +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.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; + +/** + * 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()); + } + + @Override + public Handle upload(LogFilePath localPath) throws Exception { + final String gsBucket = mConfig.getGsBucket(); + final String gsKey = localPath.withPrefix(mConfig.getGsPath()).getLogFilePath(); + File localFile = new File(localPath.getLogFilePath()); + + 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 { + mClient.objects().insert(gsBucket, storageObject, storageContent).execute(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + + return new FutureHandle(f); + } + + private static Storage getService(String credentialsPath) 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) { + 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, credential) + .setApplicationName("com.pinterest.secor") + .build(); + } + return mStorageService; + } + +} From df9d5a4aa1ab2b602a4d4484c4ffe4d5d8921fbb Mon Sep 17 00:00:00 2001 From: Space Date: Thu, 10 Sep 2015 11:49:34 -0700 Subject: [PATCH 086/330] Remove s3a testing. --- src/main/scripts/run_tests.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 89562f9f5..07e721183 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -54,7 +54,7 @@ READER_WRITERS[json]=com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFa READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory # Hadoop supports multiple implementations of the s3 filesytem -S3_FILESYSTEMS=${S3_FILESYSTEMS:-s3a s3n} +S3_FILESYSTEMS=${S3_FILESYSTEMS:-s3n} # 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. From a8b7bbd8d86b86d947e78226cd3c02ddb3d30d3e Mon Sep 17 00:00:00 2001 From: Space Date: Thu, 10 Sep 2015 18:13:29 -0700 Subject: [PATCH 087/330] Lower wait time to speed up tests. --- src/main/scripts/run_tests.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 07e721183..3c09c4042 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -56,9 +56,9 @@ READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriterFacto # Hadoop supports multiple implementations of the s3 filesytem S3_FILESYSTEMS=${S3_FILESYSTEMS:-s3n} -# 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=${SECOR_WAIT_TIME:-120} +# 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}/.. @@ -365,8 +365,9 @@ move_offset_back_test() { set_offsets_in_zookeeper 2 post_messages $((${MESSAGES}*9/10)) 0 - echo "Waiting $((${WAIT_TIME}*2)) sec for Secor to upload logs to s3" - sleep $((${WAIT_TIME}*2)) + # 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 From 2a0cb58dbd9227f88a9a66a56160203c30e0df17 Mon Sep 17 00:00:00 2001 From: Jerome Gagnon Date: Tue, 15 Sep 2015 16:09:07 -0400 Subject: [PATCH 088/330] Use direct upload for Google Storage --- .../secor/uploader/GsUploadManager.java | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java index 9d0a54bea..46bfed191 100644 --- a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java @@ -2,6 +2,8 @@ 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.HttpTransport; import com.google.api.client.json.JsonFactory; @@ -25,7 +27,7 @@ /** * 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. @@ -60,7 +62,7 @@ public GsUploadManager(SecorConfig config) throws Exception { public Handle upload(LogFilePath localPath) throws Exception { final String gsBucket = mConfig.getGsBucket(); final String gsKey = localPath.withPrefix(mConfig.getGsPath()).getLogFilePath(); - File localFile = new File(localPath.getLogFilePath()); + final File localFile = new File(localPath.getLogFilePath()); LOG.info("uploading file {} to gs://{}/{}", localFile, gsBucket, gsKey); @@ -71,7 +73,17 @@ public Handle upload(LogFilePath localPath) throws Exception { @Override public void run() { try { - mClient.objects().insert(gsBucket, storageObject, storageContent).execute(); + Storage.Objects.Insert request = mClient.objects().insert(gsBucket, storageObject, storageContent); + // Use direct upload as we are not handling resuming and it seems to be more stable. + 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); } @@ -89,7 +101,8 @@ private static Storage getService(String credentialsPath) throws Exception { try { // Lookup if configured path from the properties; otherwise fallback to Google Application default if (credentialsPath != null) { - credential = GoogleCredential.fromStream(new FileInputStream(credentialsPath), httpTransport, JSON_FACTORY) + credential = GoogleCredential + .fromStream(new FileInputStream(credentialsPath), httpTransport, JSON_FACTORY) .createScoped(Collections.singleton(StorageScopes.CLOUD_PLATFORM)); } else { credential = GoogleCredential.getApplicationDefault(httpTransport, JSON_FACTORY); From b9b49d19b190574a0a6867c0dab1101d61f60d3f Mon Sep 17 00:00:00 2001 From: Jerome Gagnon Date: Wed, 23 Sep 2015 12:08:36 -0400 Subject: [PATCH 089/330] Backoff handler for google upload --- README.md | 1 + .../pinterest/secor/common/SecorConfig.java | 8 ++++ .../secor/uploader/GsUploadManager.java | 37 ++++++++++++++++--- 3 files changed, 41 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index ec96b3c62..051b4dffc 100644 --- a/README.md +++ b/README.md @@ -129,6 +129,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Strava](https://www.strava.com) * [TiVo](https://www.tivo.com) * [Yelp](http://www.yelp.com) + * [VarageSale](http://www.varagesale.com) ## Help diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 97433e342..f7f4f3c9a 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -272,6 +272,14 @@ 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 getBoolean(String name, boolean defaultValue) { return mProperties.getBoolean(name, defaultValue); } diff --git a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java index 46bfed191..4fe4894c2 100644 --- a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java @@ -5,9 +5,14 @@ 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; @@ -55,7 +60,8 @@ public class GsUploadManager extends UploadManager { public GsUploadManager(SecorConfig config) throws Exception { super(config); - mClient = getService(mConfig.getGsCredentialsPath()); + mClient = getService(mConfig.getGsCredentialsPath(), + mConfig.getGsConnectTimeoutInMs(), mConfig.getGsReadTimeoutInMs()); } @Override @@ -74,8 +80,7 @@ public Handle upload(LogFilePath localPath) throws Exception { public void run() { try { Storage.Objects.Insert request = mClient.objects().insert(gsBucket, storageObject, storageContent); - // Use direct upload as we are not handling resuming and it seems to be more stable. - request.getMediaHttpUploader().setDirectUploadEnabled(true); + request.getMediaHttpUploader().setProgressListener(new MediaHttpUploaderProgressListener() { @Override public void progressChanged(MediaHttpUploader uploader) throws IOException { @@ -83,6 +88,7 @@ public void progressChanged(MediaHttpUploader uploader) throws IOException { (int) uploader.getProgress() * 100, localFile, gsBucket, gsKey); } }); + request.execute(); } catch (IOException e) { throw new RuntimeException(e); @@ -93,7 +99,7 @@ public void progressChanged(MediaHttpUploader uploader) throws IOException { return new FutureHandle(f); } - private static Storage getService(String credentialsPath) throws Exception { + private static Storage getService(String credentialsPath, int connectTimeoutMs, int readTimeoutMs) throws Exception { if (mStorageService == null) { HttpTransport httpTransport = GoogleNetHttpTransport.newTrustedTransport(); @@ -111,11 +117,32 @@ private static Storage getService(String credentialsPath) throws Exception { throw new RuntimeException("Failed to load Google credentials : " + credentialsPath, e); } - mStorageService = new Storage.Builder(httpTransport, JSON_FACTORY, credential) + 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); + } + }; + } + } From a9461937e9d1fa07dc4cf1fce34e9c25d9cf3be2 Mon Sep 17 00:00:00 2001 From: Baishampayan Ghose Date: Fri, 25 Sep 2015 12:47:24 +0530 Subject: [PATCH 090/330] Allow setting a custom file extension to files. Set the config param `secor.file.extension` to customize the log file extension. eg: secor.file.extension=.bin --- src/main/config/secor.common.properties | 4 ++++ src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../java/com/pinterest/secor/parser/PartitionFinalizer.java | 4 +++- 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 5731f1b33..9a5178e9b 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -159,6 +159,10 @@ message.timestamp.input.pattern= # '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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index f7f4f3c9a..d13afeab6 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -184,6 +184,10 @@ 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"); } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 204034bae..eb87858df 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -54,7 +54,9 @@ public PartitionFinalizer(SecorConfig config) throws Exception { mMessageParser = (TimestampedMessageParser) ReflectionUtil.createMessageParser( mConfig.getMessageParserClass(), mConfig); mQuboleClient = new QuboleClient(mConfig); - if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + if (mConfig.getFileExtension() != null) { + mFileExtension = mConfig.getFileExtension(); + } else if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { CompressionCodec codec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); mFileExtension = codec.getDefaultExtension(); } else { From 7e545e4b6e18f11f6e3825e503b00673d5c2faea Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Fri, 25 Sep 2015 22:19:02 -0700 Subject: [PATCH 091/330] Adding support for RegEx based parsing. We have cases where the kafka message is a plain text and people use regex to extract the fields (e.g. timestamp). Add support to allow people specify RegEx based parser, "message.timestamp.input.pattern" is used to specify a regex pattern to extract the timestamp, e.g.: ^[^ ]+ [^ ]+ ([^ ]+) .*$ The first group in the regex indicates the timestamp field --- .../secor/parser/RegexMessageParser.java | 56 +++++++++++++++ .../secor/parser/RegexMessageParserTest.java | 70 +++++++++++++++++++ 2 files changed, 126 insertions(+) create mode 100644 src/main/java/com/pinterest/secor/parser/RegexMessageParser.java create mode 100644 src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java 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..332f346a8 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java @@ -0,0 +1,56 @@ +/** + * 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 java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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. + */ +public class RegexMessageParser extends TimestampedMessageParser { + private static final Logger LOG = LoggerFactory.getLogger(RegexMessageParser.class); + + private final Pattern tsPattern; + + public RegexMessageParser(SecorConfig config) { + super(config); + String patStr = config.getMessageTimestampInputPattern(); + LOG.info("timestamp pattern: " + patStr); + tsPattern = Pattern.compile(patStr); + } + + @Override + public long extractTimestampMillis(final Message message) { + String line = new String(message.getPayload()); + Matcher m = tsPattern.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/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java new file mode 100644 index 000000000..79884813e --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.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; + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("^[^ ]+ [^ ]+ ([^ ]+) .*$"); + + 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, messageWithMillisTimestamp); + + 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, messageWithWrongFormatTimestamp); + + } + + @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, emptyBytes2)); + } + + @Test(expected=NumberFormatException.class) + public void testExtractTimestampMillisException1() throws Exception { + RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); + regexMessageParser.extractTimestampMillis(mMessageWithWrongFormatTimestamp); + } + +} From 0121f1d58bd94f7fc42b678cf7175d1a1036895e Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Fri, 25 Sep 2015 22:19:02 -0700 Subject: [PATCH 092/330] Adding support for RegEx based parsing. We have cases where the kafka message is a plain text and people use regex to extract the fields (e.g. timestamp). Add support to allow people specify RegEx based parser, "message.timestamp.input.pattern" is used to specify a regex pattern to extract the timestamp, e.g.: ^[^ ]+ [^ ]+ ([^ ]+) .*$ The first group in the regex indicates the timestamp field --- .../secor/parser/RegexMessageParser.java | 58 +++++++++++++++ .../secor/parser/RegexMessageParserTest.java | 70 +++++++++++++++++++ 2 files changed, 128 insertions(+) create mode 100644 src/main/java/com/pinterest/secor/parser/RegexMessageParser.java create mode 100644 src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java 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..3574ae046 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.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.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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); + } + + @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/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java new file mode 100644 index 000000000..79884813e --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.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; + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("^[^ ]+ [^ ]+ ([^ ]+) .*$"); + + 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, messageWithMillisTimestamp); + + 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, messageWithWrongFormatTimestamp); + + } + + @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, emptyBytes2)); + } + + @Test(expected=NumberFormatException.class) + public void testExtractTimestampMillisException1() throws Exception { + RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); + regexMessageParser.extractTimestampMillis(mMessageWithWrongFormatTimestamp); + } + +} From 049379a6168eb56c688b387896bb4462aa2e76e1 Mon Sep 17 00:00:00 2001 From: Guy Hadash Date: Tue, 24 Feb 2015 14:07:58 +0200 Subject: [PATCH 093/330] Openstack Swift integration --- README.md | 3 +- pom.xml | 9 ++- src/main/config/secor.common.properties | 18 +++++ src/main/config/secor.dev.backup.properties | 3 + .../config/secor.dev.partition.properties | 3 + src/main/config/secor.dev.properties | 17 +++++ src/main/config/secor.prod.backup.properties | 5 +- .../config/secor.prod.partition.properties | 3 + src/main/config/secor.prod.properties | 22 ++++++- .../pinterest/secor/common/SecorConfig.java | 48 ++++++++++++++ .../secor/parser/PartitionFinalizer.java | 5 +- .../secor/tools/LogFileVerifier.java | 8 +-- .../secor/uploader/HadoopS3UploadManager.java | 9 +-- .../pinterest/secor/uploader/Uploader.java | 2 +- .../com/pinterest/secor/util/FileUtil.java | 65 +++++++++++++++---- src/main/scripts/run_tests.sh | 48 ++++++++++---- .../secor/uploader/UploaderTest.java | 10 ++- .../pinterest/secor/util/FileUtilTest.java | 58 +++++++++++++++++ 18 files changed, 291 insertions(+), 45 deletions(-) create mode 100644 src/test/java/com/pinterest/secor/util/FileUtilTest.java diff --git a/README.md b/README.md index ec96b3c62..20dba9fe0 100644 --- a/README.md +++ b/README.md @@ -2,7 +2,7 @@ [![Build Status](https://travis-ci.org/pinterest/secor.svg)](https://travis-ci.org/pinterest/secor) -Secor is a service persisting [Kafka] logs to [Amazon S3]. +Secor is a service persisting [Kafka] logs to [Amazon S3] 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, @@ -143,3 +143,4 @@ If you have any questions or comments, you can reach us at [secor-users@googlegr [OpenTSDB]: http://opentsdb.net/ [Qubole]: http://www.qubole.com/ [statsD]: https://github.com/etsy/statsd/ +[Openstack Swift]: http://swift.openstack.org diff --git a/pom.xml b/pom.xml index c06eca60d..f3ad487a4 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ 0.2-SNAPSHOT jar secor - Kafka to s3/gs logs exporter + Kafka to s3/gs/swift logs exporter https://github.com/pinterest/secor @@ -151,6 +151,7 @@ commons-configuration 1.9 + org.apache.hadoop hadoop-common @@ -179,6 +180,12 @@ + + org.apache.hadoop + hadoop-openstack + 2.7.0 + + org.apache.thrift libthrift diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 5731f1b33..9c186fa48 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -20,11 +20,29 @@ # Regular expression matching names of consumed topics. secor.kafka.topic_filter=.* +# Choose what to fill according to the service you are using +# in the choice option you MUST fill S3 or Swift +cloud.service=S3 + # AWS authentication credentials. # Leave empty if using IAM role-based authentication with s3a filesystem. aws.access.key= aws.secret.key= +# 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= + ################ # 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.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 ec30afa71..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 # ################ 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 56047cb57..5c119681b 100644 --- a/src/main/config/secor.prod.properties +++ b/src/main/config/secor.prod.properties @@ -19,9 +19,6 @@ 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= @@ -29,6 +26,25 @@ kafka.seed.broker.host= # List of Kafka Zookeeper servers. 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 # ################ diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 97433e342..886feb231 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -137,6 +137,18 @@ public int getMessagesPerSecond() { public String getS3FileSystem() { return getString("secor.s3.filesystem"); } + public boolean getSeperateContainersForTopics() { + 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"); } @@ -188,6 +200,10 @@ public int getOstrichPort() { return getInt("ostrich.port"); } + public String getCloudService() { + return getString("cloud.service"); + } + public String getAwsAccessKey() { return getString("aws.access.key"); } @@ -204,6 +220,38 @@ public String getAwsRegion() { return getString("aws.region"); } + 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"); } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 204034bae..0d08dbc73 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -86,6 +86,7 @@ private String[] getFinalizedUptoPartitions(String topic) throws Exception { } 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); @@ -96,7 +97,7 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw // 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(mConfig.getS3Prefix(), topic, previous, + LogFilePath logFilePath = new LogFilePath(prefix, topic, previous, mConfig.getGeneration(), 0, 0, mFileExtension); String logFileDir = logFilePath.getLogFileDir(); if (FileUtil.exists(logFileDir)) { @@ -163,7 +164,7 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw } // Generate the SUCCESS file at the end - LogFilePath logFilePath = new LogFilePath(mConfig.getS3Prefix(), topic, current, + LogFilePath logFilePath = new LogFilePath(prefix, topic, current, mConfig.getGeneration(), 0, 0, mFileExtension); String logFileDir = logFilePath.getLogFileDir(); String successFilePath = logFileDir + "/_SUCCESS"; diff --git a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java index 348eda740..4548e1e25 100644 --- a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java +++ b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java @@ -48,12 +48,12 @@ public LogFileVerifier(SecorConfig config, String topic) throws IOException { new HashMap>>(); } - private String getTopicPrefix() { - return mConfig.getS3Prefix() + "/" + mTopic; + private String getTopicPrefix() throws IOException { + return FileUtil.getPrefix(mTopic, mConfig) + "/" + mTopic; } private void populateTopicPartitionToOffsetToFiles() throws IOException { - String prefix = mConfig.getS3Prefix(); + String prefix = FileUtil.getPrefix(mTopic, mConfig); String topicPrefix = getTopicPrefix(); String[] paths = FileUtil.listRecursively(topicPrefix); for (String path : paths) { @@ -210,4 +210,4 @@ private FileReader createFileReader(LogFilePath logFilePath) throws Exception { ); return fileReader; } -} \ No newline at end of file +} diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java index 22989d6e1..59be519ff 100644 --- a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -44,16 +44,17 @@ public HadoopS3UploadManager(SecorConfig config) { } public Handle upload(LogFilePath localPath) throws Exception { - LogFilePath s3Path = localPath.withPrefix(mConfig.getS3Prefix()); + String prefix = FileUtil.getPrefix(localPath.getTopic(), mConfig); + LogFilePath path = localPath.withPrefix(prefix); final String localLogFilename = localPath.getLogFilePath(); - final String s3LogFilename = s3Path.getLogFilePath(); - LOG.info("uploading file {} to {}", localLogFilename, s3LogFilename); + final String logFilename = path.getLogFilePath(); + LOG.info("uploading file {} to {}", localLogFilename, logFilename); final Future f = executor.submit(new Runnable() { @Override public void run() { try { - FileUtil.moveToS3(localLogFilename, s3LogFilename); + FileUtil.moveToCloud(localLogFilename, logFilename); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index ef9c8e3e2..3827ee8b1 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -36,7 +36,7 @@ /** * 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) */ diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index 0e5784b31..7946a48bb 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -38,15 +38,30 @@ public class FileUtil { public static void configure(SecorConfig config) { if (config != null) { - 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()); + 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.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()); + } } } } @@ -55,6 +70,28 @@ public static FileSystem getFileSystem(String path) throws IOException { return FileSystem.get(URI.create(path), mConf); } + public static String getPrefix(String topic, SecorConfig config) throws IOException { + String prefix = null; + if (config.getCloudService().equals("Swift")) { + String container = null; + if (config.getSeperateContainersForTopics()) { + 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 { + prefix = "s3n://" + config.getS3Bucket() + "/" + config.getS3Path(); + } + return prefix; + } + + public static String[] list(String path) throws IOException { FileSystem fs = getFileSystem(path); Path fsPath = new Path(path); @@ -63,7 +100,7 @@ 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://") || path.startsWith("s3a://")) { + if (path.startsWith("s3://") || path.startsWith("s3n://") || path.startsWith("s3a://") || path.startsWith("swift://")) { paths.add(statusPath.toUri().toString()); } else { paths.add(statusPath.toUri().getPath()); @@ -104,10 +141,10 @@ public static void delete(String path) throws IOException { } } - public static void moveToS3(String srcLocalPath, String dstS3Path) throws IOException { + 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 { @@ -126,7 +163,7 @@ 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://") || path.startsWith("s3a://")) { + if (path.startsWith("s3://") || path.startsWith("s3n://") || path.startsWith("s3a://") || path.startsWith("swift://")) { stringPath = statusPath.toUri().toString(); } else { stringPath = statusPath.toUri().getPath(); diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 3c09c4042..6d0217dfe 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -39,6 +39,9 @@ PARENT_DIR=/tmp/secor_dev LOGS_DIR=${PARENT_DIR}/logs 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. @@ -62,6 +65,11 @@ WAIT_TIME=${SECOR_WAIT_TIME:-40} BASE_DIR=$(dirname $0) CONF_DIR=${BASE_DIR}/.. +cloudService="s3" +if [ "$#" != "0" ]; then + cloudService=${1} +fi + source ${BASE_DIR}/run_common.sh run_command() { @@ -79,12 +87,22 @@ check_for_native_libs() { recreate_dirs() { run_command "rm -r -f ${PARENT_DIR}" - 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}" + 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 - run_command "s3cmd del --recursive ${S3_LOGS_DIR}" - run_command "s3cmd ls -r ${S3_LOGS_DIR}" + 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 @@ -204,10 +222,14 @@ verify() { fi # Verify SUCCESS file - 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" + if [ ${cloudService} = "swift" ]; then + run_command "swift list ${SWIFT_CONTAINER} | 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" + 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=$( logFilePaths = new HashSet(); logFilePaths.add(mLogFilePath); @@ -132,13 +135,14 @@ public void testUploadFiles() throws Exception { 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( + 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/" 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..1d2a1b7bb --- /dev/null +++ b/src/test/java/com/pinterest/secor/util/FileUtilTest.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.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 mS3Config; + + @Before + public void setUp() throws Exception { + mSwiftConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mSwiftConfig.getCloudService()).thenReturn("Swift"); + Mockito.when(mSwiftConfig.getSeperateContainersForTopics()).thenReturn(false); + Mockito.when(mSwiftConfig.getSwiftContainer()).thenReturn("some_container"); + Mockito.when(mSwiftConfig.getSwiftPath()).thenReturn("some_swift_parent_dir"); + + mS3Config = Mockito.mock(SecorConfig.class); + Mockito.when(mS3Config.getCloudService()).thenReturn("S3"); + Mockito.when(mS3Config.getS3Bucket()).thenReturn("some_bucket"); + Mockito.when(mS3Config.getS3Path()).thenReturn("some_s3_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(mS3Config); + Assert.assertEquals(FileUtil.getPrefix("some_topic", mS3Config), + "s3n://some_bucket/some_s3_parent_dir"); + + // return to the previous state + FileUtil.configure(null); + } +} From 5c4a3b173ae9c658c3cb34177a05e735c830ff47 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 29 Sep 2015 13:53:07 -0700 Subject: [PATCH 094/330] Change to the version to 0.2 to cut a 0.2 version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f3ad487a4..0d33bac73 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.2-SNAPSHOT + 0.2 jar secor Kafka to s3/gs/swift logs exporter From 6e73c269f1e3e064a03439eb82b2d2363a1a901b Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 29 Sep 2015 14:21:36 -0700 Subject: [PATCH 095/330] Bump the version to 0.3-SNAPSHOT to go back to snapshot mode --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0d33bac73..48be0ac3b 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.2 + 0.3-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From f324afe5349f739521a4af56259c740bad7a69ec Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 30 Sep 2015 13:45:59 -0700 Subject: [PATCH 096/330] Change the rebalance partition assignment from roundrobin back to range When secor was packaged with kafka811, the partition assignment between consumers threads is range assignment (this is the only assignment supported in 811). When yyejun upgrades secor with kafka821, https://github.com/pinterest/secor/commit/42ab0a61ea1a4353b614cb2ff37638fc1893d658 he changed the partition assignment from 'range' to 'roundrobin'), this has two issues on us: 1. when we migrate secor's version, we cannot do rolling upgrade anymore since the partition assignment was done differently between the old and new nodes; 2. I feel the range assignment is probably better for us, in the range assignment you can pre-calculate the assignment map by hand too, 'roundrobin' is difficult because it sorted the partition by partition's hashcode. In our secor deployment, sometimes we see some kafka partitions don't have consumer owners, if we have easy-to-calculate map, we can quickly narrow down to a particular secor host to check it's log. The two partition assignment code can be found in https://apache.googlesource.com/kafka/+/4408f487ae911e29dbc6ef4d94010a6fad702109/core/src/main/scala/kafka/consumer/PartitionAssignor.scala We are reverting the partition assignment from roundrobin back to range. But leave a configuration parameter override if some people wants the roundrobin behavior. --- src/main/config/secor.common.properties | 6 ++++++ src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ src/main/java/com/pinterest/secor/reader/MessageReader.java | 2 +- 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 4503dac61..3e7178ad0 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -73,6 +73,12 @@ secor.zookeeper.path=/ # Impacts how frequently the upload logic is triggered if no messages are delivered. kafka.consumer.timeout.ms=10000 +# 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= diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 1cb5ac3bc..0cfa3f6ac 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -87,6 +87,10 @@ public int getConsumerTimeoutMs() { return getInt("kafka.consumer.timeout.ms"); } + public String getPartitionAssignmentStrategy() { + return getString("kafka.partition.assignment.strategy"); + } + public String getRebalanceMaxRetries() { return getString("kafka.rebalance.max.retries"); } diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index cfa202548..146979288 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -115,7 +115,7 @@ private ConsumerConfig createConsumerConfig() throws UnknownHostException { props.put("auto.offset.reset", "smallest"); props.put("consumer.timeout.ms", Integer.toString(mConfig.getConsumerTimeoutMs())); props.put("consumer.id", IdUtil.getConsumerId()); - props.put("partition.assignment.strategy", "roundrobin"); + props.put("partition.assignment.strategy", mConfig.getPartitionAssignmentStrategy()); if (mConfig.getRebalanceMaxRetries() != null && !mConfig.getRebalanceMaxRetries().isEmpty()) { props.put("rebalance.max.retries", mConfig.getRebalanceMaxRetries()); From 5f3d731539f051342e5bc9d48b8f4b41bf452c11 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 30 Sep 2015 14:15:42 -0700 Subject: [PATCH 097/330] Update the version to 0.3 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 48be0ac3b..8388a19df 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.3-SNAPSHOT + 0.4 jar secor Kafka to s3/gs/swift logs exporter From 80eeb595b68b60b3b7bdaae996d9fb54fee9fb12 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 30 Sep 2015 14:16:46 -0700 Subject: [PATCH 098/330] Going back to SNAPSHOT version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8388a19df..0817f65d3 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.4 + 0.4-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 0d580600d426ce9a71211d499ec7b811f8c2950d Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 30 Sep 2015 14:20:45 -0700 Subject: [PATCH 099/330] We accidentally published 0.4 version to maven instead of 0.3, for now we just have to move forward with 0.5-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0817f65d3..4f70056cd 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.4-SNAPSHOT + 0.5-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From a40d2456d474291adabc542970811b8eac16bc3f Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 13 Oct 2015 16:59:36 -0700 Subject: [PATCH 100/330] Adding a shutdown hook to clean up the local files before exiting. When secor process died, often we found there are a large set of local data files left on the disk. Although we have a cleanup thread to clean files older than 24 hours, but sometimes this is already too late. Add a shutdown hook to clean up files on process exit. Changed a log message from debug to warning to help the debug. --- .../pinterest/secor/consumer/Consumer.java | 15 ++++++----- .../pinterest/secor/main/ConsumerMain.java | 26 ++++++++++++++++++- .../pinterest/secor/writer/MessageWriter.java | 4 +-- 3 files changed, 36 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 34155fa25..b853ba28e 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -51,6 +51,7 @@ public class Consumer extends Thread { private SecorConfig mConfig; + private FileRegistry mFileRegistry; private MessageReader mMessageReader; private MessageWriter mMessageWriter; private MessageParser mMessageParser; @@ -66,11 +67,11 @@ public Consumer(SecorConfig config) { private void init() throws Exception { mOffsetTracker = new OffsetTracker(); mMessageReader = new MessageReader(mConfig, mOffsetTracker); - FileRegistry fileRegistry = new FileRegistry(mConfig); + mFileRegistry = new FileRegistry(mConfig); UploadManager uploadManager = ReflectionUtil.createUploadManager(mConfig.getUploadManagerClass(), mConfig); - mUploader = new Uploader(mConfig, mOffsetTracker, fileRegistry, uploadManager); - mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, fileRegistry); + mUploader = new Uploader(mConfig, mOffsetTracker, mFileRegistry, uploadManager); + mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, mFileRegistry); mMessageParser = ReflectionUtil.createMessageParser(mConfig.getMessageParserClass(), mConfig); mUnparsableMessages = 0.; } @@ -159,12 +160,14 @@ private boolean consumeNextMessage() { return true; } + public FileRegistry getFileRegistry() { + return mFileRegistry; + } + /** * Helper to get the offset tracker (used in tests) * - * @param topic - * @param partition - * @return + * @return the OffsetTracker */ public OffsetTracker getOffsetTracker() { return this.mOffsetTracker; diff --git a/src/main/java/com/pinterest/secor/main/ConsumerMain.java b/src/main/java/com/pinterest/secor/main/ConsumerMain.java index b328cefad..35d448b65 100644 --- a/src/main/java/com/pinterest/secor/main/ConsumerMain.java +++ b/src/main/java/com/pinterest/secor/main/ConsumerMain.java @@ -16,8 +16,10 @@ */ package com.pinterest.secor.main; +import com.pinterest.secor.common.FileRegistry; 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; @@ -60,14 +62,36 @@ public static void main(String[] args) { logFileDeleter.deleteOldLogs(); RateLimitUtil.configure(config); + final LinkedList consumers = new LinkedList(); + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run () { + LOG.info("Shutdown hook invoked"); + for (Consumer consumer : consumers) { + FileRegistry fileRegistry = consumer.getFileRegistry(); + if (fileRegistry != null) { + for (TopicPartition tp : fileRegistry.getTopicPartitions()) { + try { + LOG.info("Removing files in: {}", tp); + fileRegistry.deleteTopicPartition(tp); + } catch (Throwable ex) { + LOG.error("Unexpected exception: {}", ex); + } + } + } + } + } + }); + Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { public void uncaughtException(Thread thread, Throwable exception) { LOG.error("Thread {} failed", thread, exception); System.exit(1); } }; + LOG.info("starting {} consumer threads", config.getConsumerThreads()); - LinkedList consumers = new LinkedList(); for (int i = 0; i < config.getConsumerThreads(); ++i) { Consumer consumer = new Consumer(config); consumer.setUncaughtExceptionHandler(handler); diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index ddc4218f8..50658a684 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -72,9 +72,9 @@ public void adjustOffset(Message message) throws IOException { 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 {} does not follow sequentially the last seen offset {}. " + + LOG.warn("offset of message does not follow sequentially the last seen offset {}. " + "Deleting files in topic {} partition {}", - message, lastSeenOffset, topicPartition.getTopic(), topicPartition.getPartition()); + lastSeenOffset, topicPartition.getTopic(), topicPartition.getPartition()); mFileRegistry.deleteTopicPartition(topicPartition); } From 875df0b77d7ed7bc86d6cb52190a048e359c9c90 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 14 Oct 2015 12:08:02 -0700 Subject: [PATCH 101/330] Revert "Adding a shutdown hook to clean up the local files before exiting." This reverts commit a40d2456d474291adabc542970811b8eac16bc3f. --- .../pinterest/secor/consumer/Consumer.java | 15 +++++------ .../pinterest/secor/main/ConsumerMain.java | 26 +------------------ .../pinterest/secor/writer/MessageWriter.java | 4 +-- 3 files changed, 9 insertions(+), 36 deletions(-) diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index b853ba28e..34155fa25 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -51,7 +51,6 @@ public class Consumer extends Thread { private SecorConfig mConfig; - private FileRegistry mFileRegistry; private MessageReader mMessageReader; private MessageWriter mMessageWriter; private MessageParser mMessageParser; @@ -67,11 +66,11 @@ public Consumer(SecorConfig config) { private void init() throws Exception { mOffsetTracker = new OffsetTracker(); mMessageReader = new MessageReader(mConfig, mOffsetTracker); - mFileRegistry = new FileRegistry(mConfig); + FileRegistry fileRegistry = new FileRegistry(mConfig); UploadManager uploadManager = ReflectionUtil.createUploadManager(mConfig.getUploadManagerClass(), mConfig); - mUploader = new Uploader(mConfig, mOffsetTracker, mFileRegistry, uploadManager); - mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, mFileRegistry); + mUploader = new Uploader(mConfig, mOffsetTracker, fileRegistry, uploadManager); + mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, fileRegistry); mMessageParser = ReflectionUtil.createMessageParser(mConfig.getMessageParserClass(), mConfig); mUnparsableMessages = 0.; } @@ -160,14 +159,12 @@ private boolean consumeNextMessage() { return true; } - public FileRegistry getFileRegistry() { - return mFileRegistry; - } - /** * Helper to get the offset tracker (used in tests) * - * @return the OffsetTracker + * @param topic + * @param partition + * @return */ public OffsetTracker getOffsetTracker() { return this.mOffsetTracker; diff --git a/src/main/java/com/pinterest/secor/main/ConsumerMain.java b/src/main/java/com/pinterest/secor/main/ConsumerMain.java index 35d448b65..b328cefad 100644 --- a/src/main/java/com/pinterest/secor/main/ConsumerMain.java +++ b/src/main/java/com/pinterest/secor/main/ConsumerMain.java @@ -16,10 +16,8 @@ */ package com.pinterest.secor.main; -import com.pinterest.secor.common.FileRegistry; 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; @@ -62,36 +60,14 @@ public static void main(String[] args) { logFileDeleter.deleteOldLogs(); RateLimitUtil.configure(config); - final LinkedList consumers = new LinkedList(); - - Runtime.getRuntime().addShutdownHook(new Thread() { - @Override - public void run () { - LOG.info("Shutdown hook invoked"); - for (Consumer consumer : consumers) { - FileRegistry fileRegistry = consumer.getFileRegistry(); - if (fileRegistry != null) { - for (TopicPartition tp : fileRegistry.getTopicPartitions()) { - try { - LOG.info("Removing files in: {}", tp); - fileRegistry.deleteTopicPartition(tp); - } catch (Throwable ex) { - LOG.error("Unexpected exception: {}", ex); - } - } - } - } - } - }); - Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { public void uncaughtException(Thread thread, Throwable exception) { LOG.error("Thread {} failed", thread, exception); System.exit(1); } }; - LOG.info("starting {} consumer threads", config.getConsumerThreads()); + LinkedList consumers = new LinkedList(); for (int i = 0; i < config.getConsumerThreads(); ++i) { Consumer consumer = new Consumer(config); consumer.setUncaughtExceptionHandler(handler); diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index 50658a684..ddc4218f8 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -72,9 +72,9 @@ public void adjustOffset(Message message) throws IOException { 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.warn("offset of message does not follow sequentially the last seen offset {}. " + + LOG.debug("offset of message {} does not follow sequentially the last seen offset {}. " + "Deleting files in topic {} partition {}", - lastSeenOffset, topicPartition.getTopic(), topicPartition.getPartition()); + message, lastSeenOffset, topicPartition.getTopic(), topicPartition.getPartition()); mFileRegistry.deleteTopicPartition(topicPartition); } From 65726a3f8f4268ce7ef7eb4d8b4354dd5518a8ad Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 14 Oct 2015 12:25:45 -0700 Subject: [PATCH 102/330] Cleanup the local secor files on process exit. Sometimes secor generates large chunks of local files when process died, although we have a cleanup script, but it only cleans up the files older than 24 hours. Sometimes we gather lots of local files before 24 hour cleanup file and this can cause filesystem full. Use File.deleteOnExit to clean up the files on exit. --- src/main/java/com/pinterest/secor/common/FileRegistry.java | 3 +++ src/main/java/com/pinterest/secor/util/FileUtil.java | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index 98b2520a2..f19896c38 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -103,6 +103,9 @@ public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) mWriters.put(path, writer); mCreationTimes.put(path, System.currentTimeMillis() / 1000L); LOG.debug("created writer for path {}", path.getLogFilePath()); + LOG.debug("Register deleteOnExit for path {}", path.getLogFilePath()); + FileUtil.deleteOnExit(path.getLogFilePath()); + FileUtil.deleteOnExit(path.getLogFileCrcPath()); } return writer; } diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index 7946a48bb..7920dc106 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import java.io.File; import java.io.IOException; import java.net.URI; import java.util.ArrayList; @@ -141,6 +142,11 @@ public static void delete(String path) 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(dstCloudPath); From 229668b4ff09a351da8c2a9a86e7fbd3cc065baa Mon Sep 17 00:00:00 2001 From: lwoessner Date: Wed, 21 Oct 2015 15:13:06 -0600 Subject: [PATCH 103/330] took out tabs --- .../com/pinterest/secor/message/Message.java | 33 ++++++++++--------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/src/main/java/com/pinterest/secor/message/Message.java b/src/main/java/com/pinterest/secor/message/Message.java index 8673dc238..fa16dff02 100644 --- a/src/main/java/com/pinterest/secor/message/Message.java +++ b/src/main/java/com/pinterest/secor/message/Message.java @@ -33,43 +33,44 @@ public class Message { private byte[] mPayload; protected String fieldsToString() { - return "topic='" + mTopic + '\'' + ", kafkaPartition=" - + mKafkaPartition + ", offset=" + mOffset + ", payload=" - + new String(mPayload); + return "topic='" + mTopic + '\'' + + ", kafkaPartition=" + mKafkaPartition + + ", offset=" + mOffset + + ", payload=" + new String(mPayload); } @Override public String toString() { - return "Message{" + fieldsToString() + '}'; + return "Message{" + fieldsToString() + '}'; } public Message(String topic, int kafkaPartition, long offset, byte[] payload) { - mTopic = topic; - mKafkaPartition = kafkaPartition; - mOffset = offset; - mPayload = payload; - if (mPayload == null) { - mPayload = new byte[0]; - } + mTopic = topic; + mKafkaPartition = kafkaPartition; + mOffset = offset; + mPayload = payload; + if (mPayload == null) { + mPayload = new byte[0]; + } } public String getTopic() { - return mTopic; + return mTopic; } public int getKafkaPartition() { - return mKafkaPartition; + return mKafkaPartition; } public long getOffset() { - return mOffset; + return mOffset; } public byte[] getPayload() { - return mPayload; + return mPayload; } public void write(OutputStream output) throws IOException { - output.write(mPayload); + output.write(mPayload); } } From 39d6cfeea82e0c6400b52b5da5ca388d10b18ed9 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 14 Oct 2015 12:25:45 -0700 Subject: [PATCH 104/330] Cleanup the local secor files on process exit. Sometimes secor generates large chunks of local files when process died, although we have a cleanup script, but it only cleans up the files older than 24 hours. Sometimes we gather lots of local files before 24 hour cleanup file and this can cause filesystem full. Use File.deleteOnExit to clean up the files on exit. --- src/main/java/com/pinterest/secor/common/FileRegistry.java | 5 +++++ src/main/java/com/pinterest/secor/util/FileUtil.java | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index 98b2520a2..bd0249487 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -103,6 +103,11 @@ public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) mWriters.put(path, writer); mCreationTimes.put(path, System.currentTimeMillis() / 1000L); 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; } diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index 7946a48bb..7920dc106 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import java.io.File; import java.io.IOException; import java.net.URI; import java.util.ArrayList; @@ -141,6 +142,11 @@ public static void delete(String path) 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(dstCloudPath); From 2870e2960381f89b2d3cd1eeec6a53b73a59208a Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Mon, 16 Nov 2015 23:17:19 -0800 Subject: [PATCH 105/330] Handle the OutOfMemoryError caused by corrupted messages Occasionaly there are corrupted messages caused by upstream job, these corrupted messages caused OOMError during thrift deserialization which stopped secor pipeline. Changed the caught exception to also Error to capture OOMError. For example, this is one of the OOM we saw: java.lang.OutOfMemoryError: Java heap space at org.apache.thrift.protocol.TBinaryProtocol.readBinary(TBinaryProtocol.java:371) at org.apache.thrift.protocol.TProtocolUtil.skip(TProtocolUtil.java:109) at org.apache.thrift.protocol.TProtocolUtil.skip(TProtocolUtil.java:131) at org.apache.thrift.protocol.TProtocolUtil.skip(TProtocolUtil.java:60) at org.apache.thrift.TDeserializer.locateField(TDeserializer.java:319) at org.apache.thrift.TDeserializer.partialDeserializeField(TDeserializer.java:237) at org.apache.thrift.TDeserializer.partialDeserializeI64(TDeserializer.java:184) at com.pinterest.secor.parser.ThriftMessageParser.extractTimestampMillis(ThriftMessageParser.java:56) at com.pinterest.secor.parser.TimestampedMessageParser.extractPartitions(TimestampedMessageParser.java:98) --- src/main/java/com/pinterest/secor/consumer/Consumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 34155fa25..2dd73bad8 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -139,7 +139,7 @@ private boolean consumeNextMessage() { parsedMessage = mMessageParser.parse(rawMessage); final double DECAY = 0.999; mUnparsableMessages *= DECAY; - } catch (Exception e) { + } catch (Throwable e) { mUnparsableMessages++; final double MAX_UNPARSABLE_MESSAGES = 1000.; if (mUnparsableMessages > MAX_UNPARSABLE_MESSAGES) { From a427d5cd0784040a081721ddb121f1d4d04d1d20 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 17 Nov 2015 10:27:19 -0800 Subject: [PATCH 106/330] Bump up the version of secor to 0.5, publish the release and then to 0.6-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4f70056cd..313e1312f 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.5-SNAPSHOT + 0.6-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From ebb75e57ceb67c5a95816285ddf3f24f26e04e3b Mon Sep 17 00:00:00 2001 From: Jerome Gagnon Date: Wed, 18 Nov 2015 14:46:31 -0500 Subject: [PATCH 107/330] Backward compatibility fix and GS compatibility fix; soften config requirements --- README.md | 3 +- src/main/config/secor.common.properties | 41 ++++++++++++------- src/main/config/secor.dev.gs.properties | 6 +-- .../com/pinterest/secor/util/FileUtil.java | 16 +++++--- .../pinterest/secor/util/FileUtilTest.java | 12 +++++- 5 files changed, 52 insertions(+), 26 deletions(-) diff --git a/README.md b/README.md index 199e4dad1..3c6295a4f 100644 --- a/README.md +++ b/README.md @@ -2,7 +2,7 @@ [![Build Status](https://travis-ci.org/pinterest/secor.svg)](https://travis-ci.org/pinterest/secor) -Secor is a service persisting [Kafka] logs to [Amazon S3] and [Openstack Swift]. +Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud 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, @@ -138,6 +138,7 @@ If you have any questions or comments, you can reach us at [secor-users@googlegr [Kafka]:http://kafka.apache.org/ [Amazon S3]:http://aws.amazon.com/s3/ [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 diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 3e7178ad0..6a03bcf8b 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -21,7 +21,7 @@ secor.kafka.topic_filter=.* # Choose what to fill according to the service you are using -# in the choice option you MUST fill S3 or Swift +# in the choice option you can fill S3, GS or Swift cloud.service=S3 # AWS authentication credentials. @@ -29,24 +29,11 @@ cloud.service=S3 aws.access.key= aws.secret.key= -# 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= - ################ # 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 @@ -63,6 +50,30 @@ aws.endpoint= # 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 + # Zookeeper config. zookeeper.session.timeout.ms=3000 zookeeper.sync.time.ms=200 diff --git a/src/main/config/secor.dev.gs.properties b/src/main/config/secor.dev.gs.properties index 5e0694772..d914462eb 100644 --- a/src/main/config/secor.dev.gs.properties +++ b/src/main/config/secor.dev.gs.properties @@ -10,13 +10,13 @@ secor.upload.manager.class=com.pinterest.secor.uploader.GsUploadManager # 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 # ################ -# Google cloud storage path where files are stored within the bucket. -secor.gs.path=data - # Application credentials configuration file # https://developers.google.com/identity/protocols/application-default-credentials secor.gs.credentials.path=google_app_credentials.json \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index 7920dc106..5567d5cf0 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -39,8 +39,8 @@ public class FileUtil { public static void configure(SecorConfig config) { if (config != null) { - if (config.getCloudService().equals("Swift")){ - mConf.set("fs.swift.impl","org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem"); + 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()); @@ -52,7 +52,7 @@ public static void configure(SecorConfig config) { } else { mConf.set("fs.swift.service.GENERICPROJECT.password", config.getSwiftPassword()); } - } else { + } 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."); @@ -86,8 +86,10 @@ public static String getPrefix(String topic, SecorConfig config) throws IOExcep container = config.getSwiftContainer(); } prefix = "swift://" + container + ".GENERICPROJECT/" + config.getSwiftPath(); - } else { + } else if (config.getCloudService().equals("S3")) { prefix = "s3n://" + config.getS3Bucket() + "/" + config.getS3Path(); + } else if (config.getCloudService().equals("GS")) { + prefix = "gs://" + config.getGsBucket() + "/" + config.getGsPath(); } return prefix; } @@ -101,7 +103,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://") || path.startsWith("s3a://") || path.startsWith("swift://")) { + 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()); @@ -169,7 +172,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://") || path.startsWith("s3a://") || path.startsWith("swift://")) { + 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(); diff --git a/src/test/java/com/pinterest/secor/util/FileUtilTest.java b/src/test/java/com/pinterest/secor/util/FileUtilTest.java index 1d2a1b7bb..de43efd6a 100644 --- a/src/test/java/com/pinterest/secor/util/FileUtilTest.java +++ b/src/test/java/com/pinterest/secor/util/FileUtilTest.java @@ -27,6 +27,7 @@ public class FileUtilTest { private SecorConfig mSwiftConfig; private SecorConfig mS3Config; + private SecorConfig mGSconfig; @Before public void setUp() throws Exception { @@ -39,7 +40,12 @@ public void setUp() throws Exception { mS3Config = Mockito.mock(SecorConfig.class); Mockito.when(mS3Config.getCloudService()).thenReturn("S3"); Mockito.when(mS3Config.getS3Bucket()).thenReturn("some_bucket"); - Mockito.when(mS3Config.getS3Path()).thenReturn("some_s3_parent_dir"); + Mockito.when(mS3Config.getS3Path()).thenReturn("some_s3_parent_dir"); + + 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 @@ -52,6 +58,10 @@ public void testGetPrefix() throws Exception { Assert.assertEquals(FileUtil.getPrefix("some_topic", mS3Config), "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); } From 15cb332bf806f2254fd83e0dfa91ea02e05bf701 Mon Sep 17 00:00:00 2001 From: Richard Lee Date: Sun, 22 Nov 2015 16:05:05 -0800 Subject: [PATCH 108/330] Archive kafka message key. Create a new SequenceFile Reader/Writer that stores a MessagePack in the key which contains both the offset and the kafka message key. A number of small API changes (primarily to Message and KeyValue) to plumb the kafka message key through from the consumer to the writer. Added a new unit test for the new Reader/Writer. --- DESIGN.md | 37 ++++ .../pinterest/secor/common/KafkaClient.java | 17 +- .../java/com/pinterest/secor/io/KeyValue.java | 29 ++- ...gePackSequenceFileReaderWriterFactory.java | 165 ++++++++++++++++++ .../impl/SequenceFileReaderWriterFactory.java | 2 +- .../com/pinterest/secor/message/Message.java | 16 +- .../secor/message/ParsedMessage.java | 4 +- .../pinterest/secor/parser/MessageParser.java | 3 +- .../pinterest/secor/reader/MessageReader.java | 3 +- .../secor/tools/LogFileVerifier.java | 4 +- .../pinterest/secor/uploader/Uploader.java | 4 +- .../pinterest/secor/writer/MessageWriter.java | 2 +- .../secor/common/LogFilePathTest.java | 2 +- ...ckSequenceFileReaderWriterFactoryTest.java | 62 +++++++ .../SequenceFileReaderWriterFactoryTest.java | 4 +- .../pinterest/secor/message/MessageTest.java | 2 +- .../secor/parser/DateMessageParserTest.java | 8 +- .../secor/parser/JsonMessageParserTest.java | 12 +- .../secor/parser/MessagePackParserTest.java | 14 +- .../parser/ProtobufMessageParserTest.java | 2 +- .../secor/parser/RegexMessageParserTest.java | 6 +- 21 files changed, 348 insertions(+), 50 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactory.java create mode 100644 src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java diff --git a/DESIGN.md b/DESIGN.md index c41ed1c47..6c0b13ca6 100644 --- a/DESIGN.md +++ b/DESIGN.md @@ -164,6 +164,43 @@ Putting this all together, a message with timestamp `` writ 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/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 421ca3f90..3b58278df 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -129,11 +129,20 @@ private Message getMessage(TopicPartition topicPartition, long offset, } 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); + } return new Message(topicPartition.getTopic(), topicPartition.getPartition(), - messageAndOffset.offset(), payloadBytes); + messageAndOffset.offset(), keyBytes, payloadBytes); } private SimpleConsumer createConsumer(String host, int port, String clientName) { diff --git a/src/main/java/com/pinterest/secor/io/KeyValue.java b/src/main/java/com/pinterest/secor/io/KeyValue.java index a8e6f11af..c7f5721a9 100644 --- a/src/main/java/com/pinterest/secor/io/KeyValue.java +++ b/src/main/java/com/pinterest/secor/io/KeyValue.java @@ -25,17 +25,30 @@ */ public class KeyValue { - private final long mKey; + private final long mOffset; + private final byte[] mKafkaKey; private final byte[] mValue; - - //constructor - public KeyValue(long key, byte[] value) { - this.mKey = key; + + // constructor + public KeyValue(long offset, byte[] value) { + this.mOffset = offset; + this.mKafkaKey = new byte[0]; this.mValue = value; } - - public long getKey() { - return this.mKey; + + // constructor + public KeyValue(long offset, byte[] kafkaKey, byte[] value) { + this.mOffset = offset; + this.mKafkaKey = kafkaKey; + this.mValue = value; + } + + public long getOffset() { + return this.mOffset; + } + + public byte[] getKafkaKey() { + return this.mKafkaKey; } public byte[] getValue() { 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..187f0f207 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactory.java @@ -0,0 +1,165 @@ +/** + * 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 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; + 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_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())); + } 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.getKafkaKey(); + // output size estimate + // 1 - map header + // 1 - message pack key + // 9 - max kafka offset + // 1 - message pack key + // 5 - max (sane) kafka key size + // N - size of kafka key + // = 17 + N + ByteArrayOutputStream out = new ByteArrayOutputStream(17 + kafkaKey.length); + MessagePacker packer = MessagePack.newDefaultPacker(out) + .packMapHeader((kafkaKey.length == 0) ? 1 : 2) + .packInt(KAFKA_MESSAGE_OFFSET) + .packLong(keyValue.getOffset()); + if (kafkaKey.length != 0) { + 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(); + } + } +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java index e009d2268..0bb56a77b 100644 --- a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java @@ -107,7 +107,7 @@ public long getLength() throws IOException { @Override public void write(KeyValue keyValue) throws IOException { - this.mKey.set(keyValue.getKey()); + this.mKey.set(keyValue.getOffset()); this.mValue.set(keyValue.getValue(), 0, keyValue.getValue().length); this.mWriter.append(this.mKey, this.mValue); } diff --git a/src/main/java/com/pinterest/secor/message/Message.java b/src/main/java/com/pinterest/secor/message/Message.java index fa16dff02..1b51b6063 100644 --- a/src/main/java/com/pinterest/secor/message/Message.java +++ b/src/main/java/com/pinterest/secor/message/Message.java @@ -27,15 +27,19 @@ */ 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; protected String fieldsToString() { return "topic='" + mTopic + '\'' + ", kafkaPartition=" + mKafkaPartition + ", offset=" + mOffset + + ", kafkaKey=" + new String(mKafkaKey) + ", payload=" + new String(mPayload); } @@ -44,13 +48,17 @@ 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) { mTopic = topic; mKafkaPartition = kafkaPartition; mOffset = offset; + mKafkaKey = kafkaKey; + if (mKafkaKey == null) { + mKafkaKey = EMPTY_BYTES; + } mPayload = payload; if (mPayload == null) { - mPayload = new byte[0]; + mPayload = EMPTY_BYTES; } } @@ -66,6 +74,10 @@ public long getOffset() { return mOffset; } + public byte[] getKafkaKey() { + return mKafkaKey; + } + public byte[] getPayload() { return 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..6e212df5c 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, + public ParsedMessage(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload, String[] mPartitions) { - super(topic, kafkaPartition, offset, payload); + super(topic, kafkaPartition, offset, kafkaKey, payload); this.mPartitions = mPartitions; } diff --git a/src/main/java/com/pinterest/secor/parser/MessageParser.java b/src/main/java/com/pinterest/secor/parser/MessageParser.java index b0770760e..1d1499358 100644 --- a/src/main/java/com/pinterest/secor/parser/MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/MessageParser.java @@ -39,7 +39,8 @@ public MessageParser(SecorConfig config) { 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); } public abstract String[] extractPartitions(Message payload) throws Exception; diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 146979288..6cf6d7717 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -153,7 +153,8 @@ public Message read() { } MessageAndMetadata kafkaMessage = mIterator.next(); Message message = new Message(kafkaMessage.topic(), kafkaMessage.partition(), - kafkaMessage.offset(), kafkaMessage.message()); + kafkaMessage.offset(), kafkaMessage.key(), + kafkaMessage.message()); TopicPartition topicPartition = new TopicPartition(message.getTopic(), message.getKafkaPartition()); updateAccessTime(topicPartition); diff --git a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java index 4548e1e25..e7f585054 100644 --- a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java +++ b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java @@ -155,8 +155,8 @@ private void getOffsets(LogFilePath logFilePath, Set offsets) throws Excep FileReader reader = createFileReader(logFilePath); KeyValue record; while ((record = reader.next()) != null) { - if (!offsets.add(record.getKey())) { - throw new RuntimeException("duplicate key " + record.getKey() + " found in file " + + if (!offsets.add(record.getOffset())) { + throw new RuntimeException("duplicate key " + record.getOffset() + " found in file " + logFilePath.getLogFilePath()); } } diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index 3827ee8b1..cc984763d 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -22,7 +22,6 @@ import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.io.KeyValue; import com.pinterest.secor.util.CompressionUtil; -import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.IdUtil; import com.pinterest.secor.util.ReflectionUtil; @@ -31,7 +30,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.*; /** @@ -137,7 +135,7 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { reader = createReader(srcPath, codec); KeyValue keyVal; while ((keyVal = reader.next()) != null) { - if (keyVal.getKey() >= startOffset) { + if (keyVal.getOffset() >= startOffset) { if (writer == null) { String localPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index ddc4218f8..d98424a93 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -88,7 +88,7 @@ public void write(ParsedMessage message) throws Exception { LogFilePath path = new LogFilePath(mLocalPrefix, mGeneration, offset, message, mFileExtension); FileWriter writer = mFileRegistry.getOrCreateWriter(path, mCodec); - writer.write(new KeyValue(message.getOffset(), message.getPayload())); + writer.write(new KeyValue(message.getOffset(), message.getKafkaKey(), message.getPayload())); LOG.debug("appended message {} to file {}. File length {}", message, path, writer.getLength()); } diff --git a/src/test/java/com/pinterest/secor/common/LogFilePathTest.java b/src/test/java/com/pinterest/secor/common/LogFilePathTest.java index 2d76b9cdf..9af8887e2 100644 --- a/src/test/java/com/pinterest/secor/common/LogFilePathTest.java +++ b/src/test/java/com/pinterest/secor/common/LogFilePathTest.java @@ -50,7 +50,7 @@ protected void setUp() throws Exception { } public void testConstructFromMessage() throws Exception { - ParsedMessage message = new ParsedMessage(TOPIC, KAFKA_PARTITION, 1000, + ParsedMessage message = new ParsedMessage(TOPIC, KAFKA_PARTITION, 1000, null, "some_payload".getBytes(), PARTITIONS); LogFilePath logFilePath = new LogFilePath(PREFIX, GENERATION, LAST_COMMITTED_OFFSET, message, ""); 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..cccdc8f70 --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java @@ -0,0 +1,62 @@ +/** + * 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})); + 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.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()); + } + + +} \ 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 index eea747a78..a74b2018a 100644 --- a/src/test/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactoryTest.java +++ b/src/test/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactoryTest.java @@ -53,10 +53,10 @@ public void testSequenceReadWriteRoundTrip() throws Exception { FileReader fileReader = factory.BuildFileReader(tempLogFilePath, null); KeyValue kvout = fileReader.next(); - assertEquals(kv1.getKey(), kvout.getKey()); + assertEquals(kv1.getOffset(), kvout.getOffset()); assertArrayEquals(kv1.getValue(), kvout.getValue()); kvout = fileReader.next(); - assertEquals(kv2.getKey(), kvout.getKey()); + assertEquals(kv2.getOffset(), kvout.getOffset()); assertArrayEquals(kv2.getValue(), kvout.getValue()); } diff --git a/src/test/java/com/pinterest/secor/message/MessageTest.java b/src/test/java/com/pinterest/secor/message/MessageTest.java index e7a4ed1ea..c02ffaea7 100644 --- a/src/test/java/com/pinterest/secor/message/MessageTest.java +++ b/src/test/java/com/pinterest/secor/message/MessageTest.java @@ -8,7 +8,7 @@ public class MessageTest { @Test public void testNullPayload() { - Message message = new Message("testTopic", 0, 123, null); + Message message = new Message("testTopic", 0, 123, null, null); System.out.println(message); // no assert necessary, just making sure it does not throw a diff --git a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index 1766f2730..13beccd55 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -44,19 +44,19 @@ public void setUp() throws Exception { 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); + mFormat1 = new Message("test", 0, 0, null, format1); 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); + mFormat2 = new Message("test", 0, 0, null, format2); 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); + mFormat3 = new Message("test", 0, 0, null, format3); 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); + mInvalidDate = new Message("test", 0, 0, null, invalidDate); getTimestamp = Mockito.when(mConfig.getMessageTimestampInputPattern()); } diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index 96c0cc687..c42c5342e 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -45,19 +45,19 @@ public void setUp() throws Exception { 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, messageWithSecondsTimestamp); + mMessageWithSecondsTimestamp = new Message("test", 0, 0, null, messageWithSecondsTimestamp); byte 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, messageWithMillisTimestamp); + mMessageWithMillisTimestamp = new Message("test", 0, 0, null, messageWithMillisTimestamp); byte 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, messageWithMillisFloatTimestamp); + mMessageWithMillisFloatTimestamp = new Message("test", 0, 0, null, messageWithMillisFloatTimestamp); 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); } @Test @@ -78,7 +78,7 @@ public void testExtractTimestampMillisException1() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); byte emptyBytes1[] = {}; - jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, emptyBytes1)); + jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes1)); } @Test(expected=ClassCastException.class) @@ -86,7 +86,7 @@ public void testExtractTimestampMillisException2() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); byte emptyBytes2[] = "".getBytes(); - jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, emptyBytes2)); + jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes2)); } @Test diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java index 0b52b6128..7da475600 100644 --- a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -48,7 +48,7 @@ public void setUp() throws Exception { HashMap mapWithSecondTimestamp = new HashMap(); mapWithSecondTimestamp.put("ts", 1405970352); - mMessageWithSecondsTimestamp = new Message("test", 0, 0, + mMessageWithSecondsTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWithSecondTimestamp)); HashMap mapWithMillisTimestamp = new HashMap(); @@ -56,7 +56,7 @@ public void setUp() throws Exception { mapWithMillisTimestamp.put("isActive", true); mapWithMillisTimestamp.put("email", "alice@example.com"); mapWithMillisTimestamp.put("age", 27); - mMessageWithMillisTimestamp = new Message("test", 0, 0, + mMessageWithMillisTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWithMillisTimestamp)); @@ -65,7 +65,7 @@ public void setUp() throws Exception { mapWithMillisFloatTimestamp.put("isActive", false); mapWithMillisFloatTimestamp.put("email", "bob@example.com"); mapWithMillisFloatTimestamp.put("age", 35); - mMessageWithMillisFloatTimestamp = new Message("test", 0, 0, + mMessageWithMillisFloatTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWithMillisFloatTimestamp)); HashMap mapWithMillisStringTimestamp = new HashMap(); @@ -73,7 +73,7 @@ public void setUp() throws Exception { mapWithMillisStringTimestamp.put("isActive", null); mapWithMillisStringTimestamp.put("email", "charlie@example.com"); mapWithMillisStringTimestamp.put("age", 67); - mMessageWithMillisStringTimestamp = new Message("test", 0, 0, + mMessageWithMillisStringTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWithMillisStringTimestamp)); } @@ -94,7 +94,7 @@ public void testExtractTimestampMillis() throws Exception { public void testMissingTimestamp() throws Exception { HashMap mapWithoutTimestamp = new HashMap(); mapWithoutTimestamp.put("email", "mary@example.com"); - Message nMessageWithoutTimestamp = new Message("test", 0, 0, + Message nMessageWithoutTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWithoutTimestamp)); mMessagePackParser.extractTimestampMillis(nMessageWithoutTimestamp); } @@ -103,7 +103,7 @@ public void testMissingTimestamp() throws Exception { 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, + Message nMessageWithUnsupportedFormatTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWitUnsupportedFormatTimestamp)); mMessagePackParser.extractTimestampMillis(nMessageWithUnsupportedFormatTimestamp); } @@ -112,7 +112,7 @@ public void testUnsupportedTimestampFormat() throws Exception { public void testNullTimestamp() throws Exception { HashMap mapWitNullTimestamp = new HashMap(); mapWitNullTimestamp.put("ts", null); - Message nMessageWithNullTimestamp = new Message("test", 0, 0, + Message nMessageWithNullTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWitNullTimestamp)); mMessagePackParser.extractTimestampMillis(nMessageWithNullTimestamp); } diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index 14a0288ca..af49a579f 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -33,7 +33,7 @@ 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, data); + return new Message("test", 0, 0, null, data); } @Override diff --git a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java index 79884813e..13edce801 100644 --- a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -39,11 +39,11 @@ public void setUp() throws Exception { 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, messageWithMillisTimestamp); + mMessageWithMillisTimestamp = new Message("test", 0, 0, null, messageWithMillisTimestamp); 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, messageWithWrongFormatTimestamp); + mMessageWithWrongFormatTimestamp = new Message("test", 0, 0, null, messageWithWrongFormatTimestamp); } @@ -58,7 +58,7 @@ public void testExtractTimestampMillis() throws Exception { public void testExtractTimestampMillisEmpty() throws Exception { RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); byte emptyBytes2[] = "".getBytes(); - regexMessageParser.extractTimestampMillis(new Message("test", 0, 0, emptyBytes2)); + regexMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes2)); } @Test(expected=NumberFormatException.class) From 2ab3f1646355c7ab95bef4d94ca19c61645c87f3 Mon Sep 17 00:00:00 2001 From: Jerome Gagnon Date: Thu, 26 Nov 2015 14:58:53 -0500 Subject: [PATCH 109/330] SimpleDateFormat are NOT thread-safe --- .../parser/TimestampedMessageParser.java | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index c0f330044..78a786bcb 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -34,15 +34,14 @@ public abstract class TimestampedMessageParser extends MessageParser implements private static final long HOUR_IN_MILLIS = 3600L * 1000L; private static final long DAY_IN_MILLIS = 3600L * 24 * 1000L; - private static final SimpleDateFormat mDtFormatter = new SimpleDateFormat("yyyy-MM-dd"); - private static final SimpleDateFormat mHrFormatter = new SimpleDateFormat("HH"); - private static final SimpleDateFormat mDtHrFormatter = new SimpleDateFormat("yyyy-MM-dd-HH"); - - static { - mDtFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); - mHrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); - mDtHrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); - } + /* + * IMPORTANT + * SimpleDateFormat are NOT thread-safe. + * Each parser needs to have their own local SimpleDateFormat or it'll cause race condition. + */ + private final SimpleDateFormat mDtFormatter; + private final SimpleDateFormat mHrFormatter; + private final SimpleDateFormat mDtHrFormatter; private final boolean mUsingHourly; @@ -50,6 +49,13 @@ public TimestampedMessageParser(SecorConfig config) { super(config); mUsingHourly = usingHourly(config); LOG.info("UsingHourly: {}", mUsingHourly); + + mDtFormatter = new SimpleDateFormat("yyyy-MM-dd"); + mDtFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + mHrFormatter = new SimpleDateFormat("HH"); + mHrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + mDtHrFormatter = new SimpleDateFormat("yyyy-MM-dd-HH"); + mDtHrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); } public abstract long extractTimestampMillis(final Message message) throws Exception; From 2e09d1b73df35d1d2068fc5aed207b3896726a5f Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Mon, 30 Nov 2015 10:29:06 -0800 Subject: [PATCH 110/330] bump the version from 0.6-SNAPSHOT to 0.7-SNAPSHOT fore secor --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 313e1312f..601a6dce4 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.6-SNAPSHOT + 0.7-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From d8ebc0478574d8cc0ee01b5e19e124406ce1b010 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 3 Dec 2015 23:07:30 -0800 Subject: [PATCH 111/330] Use the strict UNIX_LINES mode to do regular expression parsing Recently we added RegexMessageParser to parse the text log file, occasionally I saw the error message about cannot parsing some messages. After investigation, I realized that the problem is some text log contain non-standard unicode (e.g. 0x85) in the url field, these 0x85 char are being treated as newline in java.text.Pattern. Fix this by using strict UNIX line parsing (i.e. only recognize 0x0a as the line terminator) --- .../java/com/pinterest/secor/parser/RegexMessageParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java index 3574ae046..11a8a41cc 100644 --- a/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java @@ -39,7 +39,7 @@ public RegexMessageParser(SecorConfig config) { super(config); String patStr = config.getMessageTimestampInputPattern(); LOG.info("timestamp pattern: {}", patStr); - mTsPattern = Pattern.compile(patStr); + mTsPattern = Pattern.compile(patStr, Pattern.UNIX_LINES); } @Override From 38a9ed531c4472ff5de5f03b1b45bf3608a20d95 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 10 Dec 2015 11:42:43 -0800 Subject: [PATCH 112/330] Upgreade version to 0.8-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 601a6dce4..0e8b3b3e1 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.7-SNAPSHOT + 0.8-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 5e5feb2f3856b7ceb7f3c39f40ea76277842479c Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Fri, 4 Dec 2015 10:24:11 -0800 Subject: [PATCH 113/330] Make the finalizer delay to be a configuration parameter We used to have hard-coded 3600 seconds delay before the partition can be finalized to allow for late-arrival events. Some use cases (e.g. hourly pull) want shorter delay time (e.g. 15 minutes). Make this parameter a configuration. --- pom.xml | 2 +- src/main/config/secor.common.properties | 3 +++ .../java/com/pinterest/secor/common/KafkaClient.java | 6 +++--- .../java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../pinterest/secor/parser/TimestampedMessageParser.java | 9 +++++++-- .../pinterest/secor/parser/JsonMessageParserTest.java | 1 + 6 files changed, 19 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 601a6dce4..0e8b3b3e1 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.7-SNAPSHOT + 0.8-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 6a03bcf8b..8d466d2b6 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -147,6 +147,9 @@ secor.topic_partition.forget.seconds=600 # The hour folder ranges from 00 to 23 # partitioner.granularity.hour=true +# 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 ...) diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 3b58278df..de6198633 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -59,7 +59,7 @@ public KafkaClient(SecorConfig config) { private HostAndPort findLeader(TopicPartition topicPartition) { SimpleConsumer consumer = null; try { - LOG.info("looking up leader for topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); + LOG.debug("looking up leader for topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); consumer = createConsumer( mConfig.getKafkaSeedBrokerHost(), mConfig.getKafkaSeedBrokerPort(), @@ -113,7 +113,7 @@ private long findLastOffset(TopicPartition topicPartition, SimpleConsumer consum private Message getMessage(TopicPartition topicPartition, long offset, SimpleConsumer consumer) { - LOG.info("fetching message topic {} partition {} offset ", + 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); @@ -151,7 +151,7 @@ private SimpleConsumer createConsumer(String host, int port, String clientName) public SimpleConsumer createConsumer(TopicPartition topicPartition) { HostAndPort leader = findLeader(topicPartition); - LOG.info("leader for topic {} partition {} is {}", topicPartition.getTopic(), topicPartition.getPartition(), leader.toString()); + LOG.debug("leader for topic {} partition {} is {}", topicPartition.getTopic(), topicPartition.getPartition(), leader.toString()); final String clientName = getClientName(topicPartition); return createConsumer(leader.getHostText(), leader.getPort(), clientName); } diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 0cfa3f6ac..d2efc31ac 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -336,6 +336,10 @@ public int getGsReadTimeoutInMs() { return getInt("secor.gs.read.timeout.ms", 3 * 60000); } + public int getFinalizerDelaySeconds() { + return getInt("partitioner.finalizer.delay.seconds"); + } + public boolean getBoolean(String name, boolean defaultValue) { return mProperties.getBoolean(name, defaultValue); } diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 78a786bcb..4b3dbafee 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -42,6 +42,7 @@ public abstract class TimestampedMessageParser extends MessageParser implements private final SimpleDateFormat mDtFormatter; private final SimpleDateFormat mHrFormatter; private final SimpleDateFormat mDtHrFormatter; + private final int mFinalizerDelaySeconds; private final boolean mUsingHourly; @@ -49,6 +50,8 @@ public TimestampedMessageParser(SecorConfig config) { super(config); mUsingHourly = usingHourly(config); LOG.info("UsingHourly: {}", mUsingHourly); + mFinalizerDelaySeconds = config.getFinalizerDelaySeconds(); + LOG.info("FinalizerDelaySeconds: {}", mFinalizerDelaySeconds); mDtFormatter = new SimpleDateFormat("yyyy-MM-dd"); mDtFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); @@ -110,7 +113,8 @@ private long getFinalizedTimestampMillis(Message lastMessage, long lastTimestamp = extractTimestampMillis(lastMessage); long committedTimestamp = extractTimestampMillis(committedMessage); long now = System.currentTimeMillis(); - if (lastTimestamp == committedTimestamp && (now - lastTimestamp) > 3600 * 1000) { + if (lastTimestamp == committedTimestamp && + (now - lastTimestamp) > mFinalizerDelaySeconds * 1000) { LOG.info("No new message coming, use the current time: " + now); return now; } @@ -132,6 +136,7 @@ public String[] getFinalizedUptoPartitions(List lastMessages, long millis = getFinalizedTimestampMillis(lastMessages.get(i), committedMessages.get(i)); if (millis < minMillis) { + LOG.info("partition {}, time {}", i, millis); minMillis = millis; } } @@ -142,7 +147,7 @@ public String[] getFinalizedUptoPartitions(List lastMessages, } // add the safety lag for late-arrival messages - minMillis -= 3600L * 1000L; + minMillis -= mFinalizerDelaySeconds * 1000L; LOG.info("adjusted millis {}", minMillis); return generatePartitions(minMillis, mUsingHourly); } diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index c42c5342e..9b75c4d1f 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -42,6 +42,7 @@ public class JsonMessageParserTest extends TestCase { public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getFinalizerDelaySeconds()).thenReturn(3600); 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"); From a75234c9ae90442c4bf3874c1e6750ee37624d41 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 10 Dec 2015 13:44:19 -0800 Subject: [PATCH 114/330] Cut 0.8 release, and bump the version to 0.9-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0e8b3b3e1..010d37b70 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.8-SNAPSHOT + 0.9-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 1a5a437bdbe28bbff8ff24e5a8d988a6a05c9787 Mon Sep 17 00:00:00 2001 From: robertfirek Date: Sat, 12 Dec 2015 15:23:59 +0000 Subject: [PATCH 115/330] Restore support for s3a file system --- .../com/pinterest/secor/util/FileUtil.java | 2 +- .../pinterest/secor/util/FileUtilTest.java | 28 ++++++++++++++----- 2 files changed, 22 insertions(+), 8 deletions(-) diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index 5567d5cf0..d46343c5d 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -87,7 +87,7 @@ public static String getPrefix(String topic, SecorConfig config) throws IOExcep } prefix = "swift://" + container + ".GENERICPROJECT/" + config.getSwiftPath(); } else if (config.getCloudService().equals("S3")) { - prefix = "s3n://" + config.getS3Bucket() + "/" + config.getS3Path(); + prefix = config.getS3Prefix(); } else if (config.getCloudService().equals("GS")) { prefix = "gs://" + config.getGsBucket() + "/" + config.getGsPath(); } diff --git a/src/test/java/com/pinterest/secor/util/FileUtilTest.java b/src/test/java/com/pinterest/secor/util/FileUtilTest.java index de43efd6a..e5f011677 100644 --- a/src/test/java/com/pinterest/secor/util/FileUtilTest.java +++ b/src/test/java/com/pinterest/secor/util/FileUtilTest.java @@ -26,7 +26,8 @@ public class FileUtilTest { private SecorConfig mSwiftConfig; - private SecorConfig mS3Config; + private SecorConfig mS3AConfig; + private SecorConfig mS3NConfig; private SecorConfig mGSconfig; @Before @@ -37,10 +38,19 @@ public void setUp() throws Exception { Mockito.when(mSwiftConfig.getSwiftContainer()).thenReturn("some_container"); Mockito.when(mSwiftConfig.getSwiftPath()).thenReturn("some_swift_parent_dir"); - mS3Config = Mockito.mock(SecorConfig.class); - Mockito.when(mS3Config.getCloudService()).thenReturn("S3"); - Mockito.when(mS3Config.getS3Bucket()).thenReturn("some_bucket"); - Mockito.when(mS3Config.getS3Path()).thenReturn("some_s3_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"); @@ -54,8 +64,12 @@ public void testGetPrefix() throws Exception { Assert.assertEquals(FileUtil.getPrefix("some_topic", mSwiftConfig), "swift://some_container.GENERICPROJECT/some_swift_parent_dir"); - //FileUtil.configure(mS3Config); - Assert.assertEquals(FileUtil.getPrefix("some_topic", mS3Config), + //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); From 14ed1fb48484bd4f8ced50b45fef28bef9ca1bde Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Mon, 14 Dec 2015 15:11:50 -0800 Subject: [PATCH 116/330] Cut 0.9 version and bump the running version to 0.10-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 010d37b70..16bef2ce8 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.9-SNAPSHOT + 0.10-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From d2a58cde4ea8dfc70f2c62fe74b0a234f2bbc114 Mon Sep 17 00:00:00 2001 From: keithdotpower Date: Wed, 9 Dec 2015 19:06:00 +0000 Subject: [PATCH 117/330] Added options for S3 managed and customer-key based encryption, fixed dependencies --- pom.xml | 16 +++++- src/main/config/secor.common.properties | 24 +++++++++ .../pinterest/secor/common/SecorConfig.java | 12 +++++ .../secor/uploader/S3UploadManager.java | 52 +++++++++++++++++-- 4 files changed, 100 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 601a6dce4..8b7939f24 100644 --- a/pom.xml +++ b/pom.xml @@ -94,7 +94,7 @@ com.amazonaws aws-java-sdk-s3 - 1.10.2 + 1.10.39 net.java.dev.jets3t @@ -178,6 +178,10 @@ com.fasterxml.jackson.core jackson-databind + + com.amazonaws + aws-java-sdk + @@ -238,6 +242,16 @@ org.msgpack jackson-dataformat-msgpack 0.7.0-p2 + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + com.timgroup diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 6a03bcf8b..e7afeede8 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -46,6 +46,30 @@ aws.secret.key= aws.region= aws.endpoint= +########################### +# 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 0cfa3f6ac..d34fe692f 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -228,6 +228,18 @@ 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"); } diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index ab5a8dbb1..85f6d1eae 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -16,9 +16,12 @@ */ 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.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.regions.Region; import com.amazonaws.regions.Regions; @@ -41,10 +44,15 @@ * default credential provider chain (supports environment variables, * system properties, credientials file, and IAM credentials). * + * If set, it will + * * @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 TransferManager mManager; @@ -77,9 +85,47 @@ public Handle upload(LogFilePath localPath) throws Exception { String s3Key = localPath.withPrefix(mConfig.getS3Path()).getLogFilePath(); File localFile = new File(localPath.getLogFilePath()); - LOG.info("uploading file {} to s3://{}/{}", localFile, s3Bucket, s3Key); + // make upload request, taking into account configured options for encryption + PutObjectRequest uploadRequest = new PutObjectRequest(s3Bucket, s3Key, localFile);; + if (mConfig.getAwsSseType() != null) { + 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(s3Bucket, s3Key, localFile); + 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); + } } From d8a71015892507a5e10083d96e7e4608a8395d61 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 15 Dec 2015 21:40:13 -0800 Subject: [PATCH 118/330] Add a new config to allow more flexible mapping from kafka topic to hive table name Currently, we have secor.hive.prefix to allow people adding prefix to a kakfa topic name when mapping to hive table name. Sometimes, people have an existing hive table name which doesn't fall in into the prefix pattern. Add new config 'secor.hive.table.name.topic1=table1' to map topic name to arbituary hive table name. This is an optional config, the existence of this property will override the hive.table.prefix property. --- src/main/config/secor.common.properties | 5 +++++ .../pinterest/secor/common/SecorConfig.java | 5 +++++ .../secor/parser/PartitionFinalizer.java | 21 ++++++++++++------- 3 files changed, 24 insertions(+), 7 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 8d466d2b6..deb87bc4c 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -171,6 +171,11 @@ qubole.api.token= # 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= diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index d2efc31ac..360ce1cba 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -296,6 +296,11 @@ 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 String getCompressionCodec() { return getString("secor.compression.codec"); } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 99ddce53a..6deaa7a42 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -150,14 +150,21 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw sb.append("'"); } LOG.info("Hive partition string: " + sb); - String hivePrefix = null; - try { - hivePrefix = mConfig.getHivePrefix(); - } catch (RuntimeException ex) { - LOG.warn("HivePrefix is not defined. Skip hive registration"); + + String hiveTableName = mConfig.getHiveTableName(topic); + LOG.info("Hive table name from config: {}", hiveTableName); + if (hiveTableName == null) { + String hivePrefix = null; + try { + hivePrefix = mConfig.getHivePrefix(); + } catch (RuntimeException ex) { + LOG.warn("HivePrefix is not defined. Skip hive registration"); + } + hiveTableName = hivePrefix + topic; + LOG.info("Hive table name from prefix: {}", hiveTableName); } - if (hivePrefix != null) { - mQuboleClient.addPartition(hivePrefix + topic, sb.toString()); + if (hiveTableName != null) { + mQuboleClient.addPartition(hiveTableName, sb.toString()); } } catch (Exception e) { LOG.error("failed to finalize topic " + topic, e); From 1c666e12e17c58086b8fe0644c342c5b2818101a Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 15 Dec 2015 22:43:52 -0800 Subject: [PATCH 119/330] Remove openjdk7 since it constantsly causing core dumps on starting zookeeper: running ./scripts/run_kafka_class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain ./scripts/../zookeeper.test.properties > /tmp/secor_dev/logs/zookeeper.log 2>&1 & *** buffer overflow detected ***: /usr/lib/jvm/java-7-openjdk-amd64/bin/java terminated ======= Backtrace: ========= /lib/x86_64-linux-gnu/libc.so.6(__fortify_fail+0x37)[0x2b029fa7ae57] --- .travis.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 263b872b6..50f212ded 100644 --- a/.travis.yml +++ b/.travis.yml @@ -5,7 +5,6 @@ addons: env: - PATH=$PATH:$HOME/.s3cmd SECOR_LOCAL_S3=true S3CMD=1.0.1 jdk: - - openjdk7 - oraclejdk7 - oraclejdk8 before_install: From 22ee03949a6e73b03c5d53d7616cf69cf0f82713 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 15 Dec 2015 21:40:13 -0800 Subject: [PATCH 120/330] Add a new config to allow more flexible mapping from kafka topic to hive table name Currently, we have secor.hive.prefix to allow people adding prefix to a kakfa topic name when mapping to hive table name. Sometimes, people have an existing hive table name which doesn't fall in into the prefix pattern. Add new config 'secor.hive.table.name.topic1=table1' to map topic name to arbituary hive table name. This is an optional config, the existence of this property will override the hive.table.prefix property. --- src/main/config/secor.common.properties | 12 ++++++++--- .../pinterest/secor/common/SecorConfig.java | 5 +++++ .../secor/parser/PartitionFinalizer.java | 21 ++++++++++++------- 3 files changed, 28 insertions(+), 10 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 8d466d2b6..78db6c2d8 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -166,11 +166,17 @@ 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 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= diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index d2efc31ac..360ce1cba 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -296,6 +296,11 @@ 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 String getCompressionCodec() { return getString("secor.compression.codec"); } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 99ddce53a..6deaa7a42 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -150,14 +150,21 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw sb.append("'"); } LOG.info("Hive partition string: " + sb); - String hivePrefix = null; - try { - hivePrefix = mConfig.getHivePrefix(); - } catch (RuntimeException ex) { - LOG.warn("HivePrefix is not defined. Skip hive registration"); + + String hiveTableName = mConfig.getHiveTableName(topic); + LOG.info("Hive table name from config: {}", hiveTableName); + if (hiveTableName == null) { + String hivePrefix = null; + try { + hivePrefix = mConfig.getHivePrefix(); + } catch (RuntimeException ex) { + LOG.warn("HivePrefix is not defined. Skip hive registration"); + } + hiveTableName = hivePrefix + topic; + LOG.info("Hive table name from prefix: {}", hiveTableName); } - if (hivePrefix != null) { - mQuboleClient.addPartition(hivePrefix + topic, sb.toString()); + if (hiveTableName != null) { + mQuboleClient.addPartition(hiveTableName, sb.toString()); } } catch (Exception e) { LOG.error("failed to finalize topic " + topic, e); From 417eaa2e424074c73880c2f7a74669a8dcc31d60 Mon Sep 17 00:00:00 2001 From: keithdotpower Date: Wed, 16 Dec 2015 08:47:15 +0000 Subject: [PATCH 121/330] Fixed up class javadoc in S3UploadManager --- .../secor/uploader/S3UploadManager.java | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index 85f6d1eae..675ec2314 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -38,13 +38,19 @@ /** * Manages uploads to S3 using the TransferManager class from the AWS * SDK. - * - * It will use the aws.access.key and aws.secret.key configuration - * settings if they are non-empty; otherwise, it will use the SDK's - * default credential provider chain (supports environment variables, - * system properties, credientials file, and IAM credentials). - * - * If set, it will + *

+ * 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) */ From 2eab729441527b570a93ae4cde2a5ffee6a82aba Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 16 Dec 2015 12:10:42 -0800 Subject: [PATCH 122/330] Cut 0.10 and bump the version to 0.11-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 16bef2ce8..b6b4b52af 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.10-SNAPSHOT + 0.11-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From e73eb2673bf0caa8055f1fa70dc28a1392b6bc94 Mon Sep 17 00:00:00 2001 From: keithdotpower Date: Thu, 17 Dec 2015 14:24:14 +0000 Subject: [PATCH 123/330] Fixed check for no AWS encryption specified --- src/main/java/com/pinterest/secor/uploader/S3UploadManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index 675ec2314..c6b09fd25 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -93,7 +93,7 @@ public Handle upload(LogFilePath localPath) throws Exception { // make upload request, taking into account configured options for encryption PutObjectRequest uploadRequest = new PutObjectRequest(s3Bucket, s3Key, localFile);; - if (mConfig.getAwsSseType() != null) { + if (!mConfig.getAwsSseType().isEmpty()) { if (S3.equals(mConfig.getAwsSseType())) { LOG.info("uploading file {} to s3://{}/{} with S3-managed encryption", localFile, s3Bucket, s3Key); enableS3Encryption(uploadRequest); From 0cbac97d352b316deb504dd6fe56b0cbb5112271 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Sun, 20 Dec 2015 20:07:59 -0800 Subject: [PATCH 124/330] Cut 0.11 and bump the running version to 0.12-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9c4f6fb57..f7c46696f 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.11-SNAPSHOT + 0.12-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From e65b79a516f50f312872bbd9918bb702c1cd5efc Mon Sep 17 00:00:00 2001 From: James Eastwood Date: Wed, 16 Dec 2015 17:39:43 +0000 Subject: [PATCH 125/330] Support timestamp fields past the first field in ThriftMessageParser This commit adds configurable functionality to ThriftmessageParser for reading a timestamp in fields other than field ID 1. Additionally timestamps in i32 fields re now supported as well as i64. Includes new properties: message.timestamp.id: (default 1) The ID or Index for the timestamp field name message.timestamp.type: (default i64) The type of the field. Rather than the previously assumed i64 the thrift parser can now read i32. (just in case you want to subject yourself to the 2038 problem). This will fall back to i64 if the value is unrecognized. Also added a test suite for the ThriftMessageParser. N.B. Benchmarks suggest that there is an immediate performance penalty just for reading past the first field, regardless of the size of the message. --- README.md | 3 +- src/main/config/secor.common.properties | 8 ++ .../pinterest/secor/common/SecorConfig.java | 8 ++ .../secor/parser/ThriftMessageParser.java | 55 ++++++++----- .../secor/parser/ThriftMessageParserTest.java | 79 +++++++++++++++++++ src/test/thrift/unittest.thrift | 18 +++++ 6 files changed, 149 insertions(+), 22 deletions(-) create mode 100644 src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java create mode 100644 src/test/thrift/unittest.thrift diff --git a/README.md b/README.md index 3c6295a4f..7db7e3d60 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ 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 1) 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 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). 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 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. @@ -130,6 +130,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [TiVo](https://www.tivo.com) * [Yelp](http://www.yelp.com) * [VarageSale](http://www.varagesale.com) + * [Skyscanner](http://www.skyscanner.net) ## Help diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 4c28b06d3..8aba8d1c3 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -218,6 +218,14 @@ statsd.hostport= # Name of field that contains timestamp for JSON, MessagePack, or Thrift message parser. (1405970352123) message.timestamp.name=timestamp +# 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= diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 455e753b5..fedec9ce0 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -296,6 +296,14 @@ public String getMessageTimestampName() { return getString("message.timestamp.name"); } + 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"); } diff --git a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java index d634baf44..b9c7b810c 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. @@ -28,33 +28,46 @@ * @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 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) { super(config); mDeserializer = new TDeserializer(); + 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/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java new file mode 100644 index 000000000..d7c9e2af6 --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java @@ -0,0 +1,79 @@ +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; + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + } + + 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); + } + + @Test + public void testExtractTimestamp() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("blasdjlkjasdkl"); + Mockito.when(mConfig.getMessageTimestampId()).thenReturn(1); + Mockito.when(mConfig.getMessageTimestampType()).thenReturn("i64"); + + 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"); + + 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"); + + 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/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 +} + From 8c2a51806947f31b4bdf27e5c793445c24eb3d99 Mon Sep 17 00:00:00 2001 From: Swapnil Marghade Date: Fri, 8 Jan 2016 12:22:47 +0530 Subject: [PATCH 126/330] Logic to read timezone from config Uncoomenting UTC default zone for test cases Changes suggested in code review in reading timezone from config. --- src/main/config/secor.common.properties | 3 ++ .../pinterest/secor/common/SecorConfig.java | 6 +++ .../parser/TimestampedMessageParser.java | 10 ++-- .../pinterest/secor/tools/LogFileDeleter.java | 3 +- .../secor/parser/JsonMessageParserTest.java | 46 +++++++++++++++++-- .../secor/parser/MessagePackParserTest.java | 2 + 6 files changed, 58 insertions(+), 12 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 8aba8d1c3..3bcf6e32e 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -250,3 +250,6 @@ 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index fedec9ce0..5f9618dd3 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -16,12 +16,14 @@ */ 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.Map; import java.util.Properties; +import java.util.TimeZone; /** * One-stop shop for Secor configuration options. @@ -365,6 +367,10 @@ public int getFinalizerDelaySeconds() { return getInt("partitioner.finalizer.delay.seconds"); } + public TimeZone getTimeZone(){ + return Strings.isNullOrEmpty(getString("secor.parser.timezone")) ? TimeZone.getTimeZone("UTC") : TimeZone.getTimeZone(getString("secor.parser.timezone")); + } + public boolean getBoolean(String name, boolean defaultValue) { return mProperties.getBoolean(name, defaultValue); } diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 4b3dbafee..e61a11d5e 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -18,14 +18,12 @@ import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; import java.text.SimpleDateFormat; import java.util.Date; -import java.util.TimeZone; +import java.util.List; public abstract class TimestampedMessageParser extends MessageParser implements Partitioner { @@ -54,11 +52,11 @@ public TimestampedMessageParser(SecorConfig config) { LOG.info("FinalizerDelaySeconds: {}", mFinalizerDelaySeconds); mDtFormatter = new SimpleDateFormat("yyyy-MM-dd"); - mDtFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + mDtFormatter.setTimeZone(config.getTimeZone()); mHrFormatter = new SimpleDateFormat("HH"); - mHrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + mHrFormatter.setTimeZone(config.getTimeZone()); mDtHrFormatter = new SimpleDateFormat("yyyy-MM-dd-HH"); - mDtHrFormatter.setTimeZone(TimeZone.getTimeZone("UTC")); + mDtHrFormatter.setTimeZone(config.getTimeZone()); } public abstract long extractTimestampMillis(final Message message) throws Exception; diff --git a/src/main/java/com/pinterest/secor/tools/LogFileDeleter.java b/src/main/java/com/pinterest/secor/tools/LogFileDeleter.java index 799ae69f2..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,7 +43,7 @@ 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); diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index 9b75c4d1f..3dd5492fc 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -16,11 +16,8 @@ */ package com.pinterest.secor.parser; -import com.pinterest.secor.common.*; +import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; - -import java.util.Arrays; -import java.util.List; import junit.framework.TestCase; import org.junit.Test; import org.junit.runner.RunWith; @@ -28,6 +25,9 @@ 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 { @@ -43,6 +43,7 @@ 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")); 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"); @@ -105,6 +106,23 @@ public void testExtractPartitions() throws Exception { 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); @@ -124,6 +142,26 @@ public void testExtractHourlyPartitions() throws Exception { 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); diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java index 7da475600..f21f56a92 100644 --- a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -28,6 +28,7 @@ import org.powermock.modules.junit4.PowerMockRunner; import java.util.HashMap; +import java.util.TimeZone; @RunWith(PowerMockRunner.class) public class MessagePackParserTest extends TestCase { @@ -43,6 +44,7 @@ public class MessagePackParserTest extends TestCase { public void setUp() throws Exception { SecorConfig mConfig = Mockito.mock(SecorConfig.class); Mockito.when(mConfig.getMessageTimestampName()).thenReturn("ts"); + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); mMessagePackParser = new MessagePackParser(mConfig); mObjectMapper = new ObjectMapper(new MessagePackFactory()); From d4c38b97947ab092094c270b11170267e1cbc6c8 Mon Sep 17 00:00:00 2001 From: Swapnil Marghade Date: Mon, 11 Jan 2016 11:53:03 +0530 Subject: [PATCH 127/330] code refactoring for long code statement --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 5f9618dd3..35fde06e6 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -367,8 +367,9 @@ public int getFinalizerDelaySeconds() { return getInt("partitioner.finalizer.delay.seconds"); } - public TimeZone getTimeZone(){ - return Strings.isNullOrEmpty(getString("secor.parser.timezone")) ? TimeZone.getTimeZone("UTC") : TimeZone.getTimeZone(getString("secor.parser.timezone")); + 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) { From c3c9b2dc745b6be858e070ff5d8d04f768d11ba9 Mon Sep 17 00:00:00 2001 From: Christian Nguyen Van Than Date: Wed, 6 Jan 2016 23:18:01 +0100 Subject: [PATCH 128/330] add 2 required properties to migrate consumer offset from zookeeper to kafka with kafka v0.9.0 add Nextperf as company who use Secor --- README.md | 1 + pom.xml | 5 +++ src/main/config/secor.common.properties | 10 ++++++ .../pinterest/secor/common/SecorConfig.java | 8 +++++ .../pinterest/secor/reader/MessageReader.java | 4 +++ .../secor.kafka.migration.test.properties | 6 ++++ .../secor/common/SecorConfigTest.java | 34 +++++++++++++++++++ 7 files changed, 68 insertions(+) create mode 100644 src/test/config/secor.kafka.migration.test.properties create mode 100644 src/test/java/com/pinterest/secor/common/SecorConfigTest.java diff --git a/README.md b/README.md index 7db7e3d60..3b1c978db 100644 --- a/README.md +++ b/README.md @@ -131,6 +131,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Yelp](http://www.yelp.com) * [VarageSale](http://www.varagesale.com) * [Skyscanner](http://www.skyscanner.net) + * [Nextperf](http://www.nextperf.com) ## Help diff --git a/pom.xml b/pom.xml index f7c46696f..981c2eb31 100644 --- a/pom.xml +++ b/pom.xml @@ -277,6 +277,11 @@ src/main/config + + + src/test/config + + org.codehaus.mojo diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 3bcf6e32e..79acda9aa 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -139,6 +139,16 @@ kafka.seed.broker.port=9092 # to as the chroot. kafka.zookeeper.path=/ +# 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 + # Secor generation is a version that should be incremented during non-backwards-compabile # Secor releases. Generation number is one of the components of generated log file names. # Generation number makes sure that outputs of different Secor versions are isolated. diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 35fde06e6..2c4dd522d 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -117,6 +117,14 @@ 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 int getGeneration() { return getInt("secor.generation"); } diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 6cf6d7717..2f8ef6d44 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -115,6 +115,10 @@ private ConsumerConfig createConsumerConfig() throws UnknownHostException { props.put("auto.offset.reset", "smallest"); props.put("consumer.timeout.ms", Integer.toString(mConfig.getConsumerTimeoutMs())); props.put("consumer.id", IdUtil.getConsumerId()); + // 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()) { 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/java/com/pinterest/secor/common/SecorConfigTest.java b/src/test/java/com/pinterest/secor/common/SecorConfigTest.java new file mode 100644 index 000000000..513763238 --- /dev/null +++ b/src/test/java/com/pinterest/secor/common/SecorConfigTest.java @@ -0,0 +1,34 @@ +package com.pinterest.secor.common; + +import org.apache.commons.configuration.ConfigurationException; +import org.apache.commons.configuration.PropertiesConfiguration; +import org.junit.Test; +import java.net.URL; +import static org.junit.Assert.*; + +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()); + } + + +} From c19181e0412f4ad491c09a9338b1a60e74bc2b2d Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 21 Jan 2016 15:03:53 -0800 Subject: [PATCH 129/330] Change the url to access to qubole from http to https, looks like qubole now requires https access now. Fix the error handling on response object to get better error report when the response is not a map. --- .../java/com/pinterest/secor/parser/QuboleClient.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/QuboleClient.java b/src/main/java/com/pinterest/secor/parser/QuboleClient.java index e39228671..cd4263ff9 100644 --- a/src/main/java/com/pinterest/secor/parser/QuboleClient.java +++ b/src/main/java/com/pinterest/secor/parser/QuboleClient.java @@ -66,7 +66,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 +86,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(); @@ -90,7 +95,7 @@ private int query(String query) throws IOException { } private void waitForCompletion(int commandId) throws IOException, InterruptedException { - URL url = new URL("http://api.qubole.com/api/v1.2/commands/" + commandId); + URL url = new URL("https://api.qubole.com/api/v1.2/commands/" + commandId); while (true) { Map response = makeRequest(url, null); if (response.get("status").equals("done")) { From 205dec139cc1df56b72d854edf74947d7099979f Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 21 Jan 2016 22:07:13 -0800 Subject: [PATCH 130/330] Cut 0.12 and Bump up the version to 0.13-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 981c2eb31..c1a0c8b08 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.12-SNAPSHOT + 0.13-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 6b93a0c4cc357e10cc32294f6a76978b6c011070 Mon Sep 17 00:00:00 2001 From: Mathias Ehlert Date: Tue, 12 Jan 2016 11:51:26 +0100 Subject: [PATCH 131/330] added support for nested timestamp fields in JSON and Date parser --- README.md | 1 + src/main/config/secor.common.properties | 6 ++++ .../pinterest/secor/common/SecorConfig.java | 4 +++ .../secor/parser/DateMessageParser.java | 2 +- .../secor/parser/JsonMessageParser.java | 2 +- .../pinterest/secor/parser/MessageParser.java | 33 +++++++++++++++++ .../secor/parser/DateMessageParserTest.java | 36 ++++++++++++++----- .../secor/parser/JsonMessageParserTest.java | 14 ++++++++ .../secor/util/ReflectionUtilTest.java | 4 +++ 9 files changed, 92 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index 3b1c978db..6af9807a1 100644 --- a/README.md +++ b/README.md @@ -132,6 +132,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [VarageSale](http://www.varagesale.com) * [Skyscanner](http://www.skyscanner.net) * [Nextperf](http://www.nextperf.com) + * [Zalando](http://www.zalando.com) ## Help diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 79acda9aa..636ca3c59 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -228,6 +228,12 @@ statsd.hostport= # 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 2c4dd522d..cd34389b3 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -305,6 +305,10 @@ public String getMonitoringPrefix() { 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"); diff --git a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java index be602046c..733483ba2 100644 --- a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java @@ -52,7 +52,7 @@ public String[] extractPartitions(Message message) { String result[] = { defaultDate }; if (jsonObject != null) { - Object fieldValue = jsonObject.get(mConfig.getMessageTimestampName()); + Object fieldValue = getJsonFieldValue(jsonObject); Object inputPattern = mConfig.getMessageTimestampInputPattern(); if (fieldValue != null && inputPattern != null) { try { diff --git a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java index 183bf7aa3..692156fc4 100644 --- a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java @@ -34,7 +34,7 @@ public JsonMessageParser(SecorConfig config) { 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) { return toMillis(Double.valueOf(fieldValue.toString()).longValue()); } diff --git a/src/main/java/com/pinterest/secor/parser/MessageParser.java b/src/main/java/com/pinterest/secor/parser/MessageParser.java index 1d1499358..36e9f83ae 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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import net.minidev.json.JSONObject; +import net.minidev.json.JSONValue; +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,9 +36,15 @@ */ public abstract class MessageParser { protected SecorConfig mConfig; + protected String[] mNestedFields; + private static final Logger LOG = LoggerFactory.getLogger(MessageParser.class); public MessageParser(SecorConfig config) { mConfig = config; + if (mConfig.getMessageTimestampName() != null && mConfig.getMessageTimestampNameSeparator() != null) { + String separatorPattern = Pattern.quote(mConfig.getMessageTimestampNameSeparator()); + mNestedFields = mConfig.getMessageTimestampName().split(separatorPattern); + } } public ParsedMessage parse(Message message) throws Exception { @@ -44,4 +55,26 @@ public ParsedMessage parse(Message message) throws Exception { } 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/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index 13beccd55..db5dfabfe 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -35,13 +35,14 @@ public class DateMessageParserTest extends TestCase { private Message mFormat2; private Message mFormat3; private Message mInvalidDate; + private Message mISOFormat; + private Message mNestedISOFormat; private OngoingStubbing getTimestamp; @Override public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); - Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); - + 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, null, format1); @@ -58,31 +59,50 @@ public void setUp() throws Exception { .getBytes("UTF-8"); mInvalidDate = new Message("test", 0, 0, null, invalidDate); - getTimestamp = Mockito.when(mConfig.getMessageTimestampInputPattern()); + byte isoFormat[] = "{\"timestamp\":\"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"); + mISOFormat = new Message("test", 0, 0, null, isoFormat); + + 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); + } @Test public void testExtractDateUsingInputPattern() throws Exception { - getTimestamp.thenReturn("yyyy-MM-dd HH:mm:ss"); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + + 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]); } @Test public void testExtractDateWithWrongEntries() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); // 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]); // 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]); } + + @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'"); + + assertEquals("dt=2016-01-11", new DateMessageParser(mConfig).extractPartitions(mNestedISOFormat)[0]); + } } diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index 3dd5492fc..25bcbe17f 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -37,6 +37,7 @@ public class JsonMessageParserTest extends TestCase { private Message mMessageWithMillisTimestamp; private Message mMessageWithMillisFloatTimestamp; private Message mMessageWithoutTimestamp; + private Message mMessageWithNestedTimestamp; @Override public void setUp() throws Exception { @@ -60,6 +61,10 @@ public void setUp() throws Exception { 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, null, messageWithoutTimestamp); + + 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); } @Test @@ -74,6 +79,15 @@ public void testExtractTimestampMillis() throws Exception { assertEquals(0l, jsonMessageParser.extractTimestampMillis(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.extractTimestampMillis(mMessageWithNestedTimestamp)); + } @Test(expected=ClassCastException.class) public void testExtractTimestampMillisException1() throws Exception { diff --git a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java index 553b58ef9..4b74650c8 100644 --- a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java +++ b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java @@ -21,14 +21,18 @@ import com.pinterest.secor.parser.MessageParser; import org.apache.commons.configuration.PropertiesConfiguration; import org.junit.Test; +import org.junit.Before; 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",""); mSecorConfig = new SecorConfig(properties); mLogFilePath = new LogFilePath("/foo", "/foo/bar/baz/1_1_1"); } From c2cc3c3320d1943ead5210ccce554ab86b7b6322 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 26 Jan 2016 11:14:35 -0800 Subject: [PATCH 132/330] Track daily and hourly stats separately Currently both daily and hourly stats are published to the same TSD metric (topic=XXX, partition=XXX}, add the third dimension (consumer_group) to differentiate the two pipelines. --- .../java/com/pinterest/secor/tools/ProgressMonitor.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 887f0cc2f..91e4c80b9 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -129,7 +129,7 @@ private void exportToTsdb(Stat stat) public void exportStats() throws Exception { List stats = getStats(); - System.out.println(JSONArray.toJSONString(stats)); + LOG.info("Stats: {}", JSONArray.toJSONString(stats)); // if there is a valid openTSDB port configured export to openTSDB if (mConfig.getTsdbHostport() != null && !mConfig.getTsdbHostport().isEmpty()) { @@ -203,7 +203,8 @@ private List getStats() throws Exception { long timestampMillisLag = lastTimestampMillis - committedTimestampMillis; Map tags = ImmutableMap.of( Stat.STAT_KEYS.TOPIC.getName(), topic, - Stat.STAT_KEYS.PARTITION.getName(), Integer.toString(partition) + Stat.STAT_KEYS.PARTITION.getName(), Integer.toString(partition), + Stat.STAT_KEYS.GROUP.getName(), mConfig.getKafkaGroup() ); long timestamp = System.currentTimeMillis() / 1000; @@ -246,7 +247,8 @@ public enum STAT_KEYS { VALUE("value"), TIMESTAMP("timestamp"), TOPIC("topic"), - PARTITION("partition"); + PARTITION("partition"), + GROUP("group"); STAT_KEYS(String name) { this.mName = name; From c558576f8d291f263b3b4c1f7e8c5f89d1d180ed Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 26 Jan 2016 11:46:17 -0800 Subject: [PATCH 133/330] Cut v0.13 and bump the running version to 0.14-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c1a0c8b08..a13ae880c 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.13-SNAPSHOT + 0.14-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 2fed2981125e0d191d61ff842dc034cf398c9fee Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 28 Jan 2016 11:08:02 -0800 Subject: [PATCH 134/330] Fix a regresion on JsonMessageParser where the timestamp separator is checked against null, not empty string This was a recent checkin by open source community, looks like the check is not complete (only checked against null, not empty string), this caused the timestamp field cannot be retrieved. --- src/main/java/com/pinterest/secor/parser/MessageParser.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/parser/MessageParser.java b/src/main/java/com/pinterest/secor/parser/MessageParser.java index 36e9f83ae..16779950d 100644 --- a/src/main/java/com/pinterest/secor/parser/MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/MessageParser.java @@ -41,7 +41,10 @@ public abstract class MessageParser { public MessageParser(SecorConfig config) { mConfig = config; - if (mConfig.getMessageTimestampName() != null && mConfig.getMessageTimestampNameSeparator() != null) { + if (mConfig.getMessageTimestampName() != null && + !mConfig.getMessageTimestampName().isEmpty() && + mConfig.getMessageTimestampNameSeparator() != null && + !mConfig.getMessageTimestampNameSeparator().isEmpty()) { String separatorPattern = Pattern.quote(mConfig.getMessageTimestampNameSeparator()); mNestedFields = mConfig.getMessageTimestampName().split(separatorPattern); } From facd2db11601fb0f45a025c2749bf1b96439884c Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 28 Jan 2016 11:56:35 -0800 Subject: [PATCH 135/330] Cut v0.14 and bump up running version to 0.15-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a13ae880c..39ec9f5f4 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.14-SNAPSHOT + 0.15-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From f4ee8f35bb92a7cde52571f987f2a744d1a1f1cd Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Mon, 1 Feb 2016 15:14:14 -0800 Subject: [PATCH 136/330] Tighten the check on missing timestamp field for json message We had a regresion last week when someone made a checkin which didn't extract the correct timestamp field which makes the json message passing through with timestamp information extracted correctly. Tighten the check to make sure the timestamp field is always required for json messages. Make this check dependdent on config property: message.timestamp.required, just in case some installations had broken messages but they don't want to change the behavior. --- src/main/config/secor.common.properties | 5 +++++ src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../java/com/pinterest/secor/parser/JsonMessageParser.java | 5 +++++ 3 files changed, 14 insertions(+) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 636ca3c59..c6ae44098 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -246,6 +246,11 @@ message.timestamp.type=i64 # Should be used when there is no timestamp in a Long format. Also ignore time zones. message.timestamp.input.pattern= +# 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'. diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index cd34389b3..576402784 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -322,6 +322,10 @@ public String getMessageTimestampInputPattern() { return getString("message.timestamp.input.pattern"); } + public boolean isMessageTimestampRequired() { + return mProperties.getBoolean("message.timestamp.required"); + } + public int getFinalizerLookbackPeriods() { return getInt("secor.finalizer.lookback.periods", 10); } diff --git a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java index 692156fc4..eab1e3c57 100644 --- a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java @@ -26,8 +26,11 @@ * 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 @@ -38,6 +41,8 @@ public long extractTimestampMillis(final Message message) { if (fieldValue != null) { return toMillis(Double.valueOf(fieldValue.toString()).longValue()); } + } else if (m_timestampRequired) { + throw new RuntimeException("Missing timestamp field for message: " + message); } return 0; } From 3e3d25a4df9558e9441adc9a64a88db1eda6c899 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Mon, 1 Feb 2016 15:50:03 -0800 Subject: [PATCH 137/330] Cut v0.15 and bump the running version to 0.16-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 39ec9f5f4..232be028c 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.15-SNAPSHOT + 0.16-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From baaca09a4a9476d83a6df55ef70255e33a44ac21 Mon Sep 17 00:00:00 2001 From: Ashish Kumar Date: Tue, 2 Feb 2016 17:21:17 +0530 Subject: [PATCH 138/330] Now user should be able to add their custom transformation logic in secor using trnasformer --- src/main/config/secor.common.properties | 3 ++ .../pinterest/secor/common/SecorConfig.java | 4 +++ .../pinterest/secor/consumer/Consumer.java | 7 +++-- .../IdentityMessageTransformer.java | 26 +++++++++++++++++ .../secor/transformer/MessageTransformer.java | 17 +++++++++++ .../pinterest/secor/util/ReflectionUtil.java | 29 ++++++++++++++++++- 6 files changed, 83 insertions(+), 3 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/transformer/IdentityMessageTransformer.java create mode 100644 src/main/java/com/pinterest/secor/transformer/MessageTransformer.java diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index c6ae44098..c7c1f6794 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -274,3 +274,6 @@ 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 576402784..6e9eae1df 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -202,6 +202,10 @@ 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"); } diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 2dd73bad8..152cc4e6f 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -22,6 +22,7 @@ import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; import com.pinterest.secor.parser.MessageParser; +import com.pinterest.secor.transformer.MessageTransformer; import com.pinterest.secor.uploader.Uploader; import com.pinterest.secor.uploader.UploadManager; import com.pinterest.secor.reader.MessageReader; @@ -55,6 +56,7 @@ public class Consumer extends Thread { private MessageWriter mMessageWriter; private MessageParser mMessageParser; private OffsetTracker mOffsetTracker; + private MessageTransformer mMessageTransformer; private Uploader mUploader; // TODO(pawel): we should keep a count per topic partition. private double mUnparsableMessages; @@ -72,6 +74,7 @@ private void init() throws Exception { mUploader = new Uploader(mConfig, mOffsetTracker, fileRegistry, uploadManager); mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, fileRegistry); mMessageParser = ReflectionUtil.createMessageParser(mConfig.getMessageParserClass(), mConfig); + mMessageTransformer = ReflectionUtil.createMessageTransformer(mConfig.getMessageTransformerClass(), mConfig); mUnparsableMessages = 0.; } @@ -136,7 +139,8 @@ private boolean consumeNextMessage() { } ParsedMessage parsedMessage = null; try { - parsedMessage = mMessageParser.parse(rawMessage); + Message transformedMessage = mMessageTransformer.transform(rawMessage); + parsedMessage = mMessageParser.parse(transformedMessage); final double DECAY = 0.999; mUnparsableMessages *= DECAY; } catch (Throwable e) { @@ -169,5 +173,4 @@ private boolean consumeNextMessage() { public OffsetTracker getOffsetTracker() { return this.mOffsetTracker; } - } 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/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index fb71cc247..2a14f7806 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -18,12 +18,13 @@ 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.FileReaderWriterFactory; import com.pinterest.secor.parser.MessageParser; +import com.pinterest.secor.transformer.MessageTransformer; import com.pinterest.secor.uploader.UploadManager; + import org.apache.hadoop.io.compress.CompressionCodec; /** @@ -134,4 +135,30 @@ public static FileReader createFileReader(String className, LogFilePath logFileP throws Exception { return createFileReaderWriterFactory(className).BuildFileReader(logFilePath, codec); } + + /** + * Create a MessageTrnasformer from it's fully qualified class name. The + * class passed in by name must be assignable to MessageTrnasformers and have + * 1-parameter constructor accepting a SecorConfig. Allows the MessageTrnasformers + * to be pluggable by providing the class name of a desired MessageTrnasformers 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); + } } From 2401df69c17f49fb860613fb717c2d0733fa3a5e Mon Sep 17 00:00:00 2001 From: vthakre Date: Wed, 3 Feb 2016 16:31:08 -0800 Subject: [PATCH 139/330] Added proxy support for S3UploadManager --- src/main/config/secor.common.properties | 6 ++++++ .../pinterest/secor/common/SecorConfig.java | 16 ++++++++++++++++ .../secor/uploader/S3UploadManager.java | 18 +++++++++++++++--- 3 files changed, 37 insertions(+), 3 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index c7c1f6794..5034021a6 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -29,6 +29,12 @@ cloud.service=S3 aws.access.key= aws.secret.key= +# 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 # ################ diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 6e9eae1df..64af04ceb 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -237,6 +237,18 @@ public String getAwsSecretKey() { public String getAwsEndpoint() { return getString("aws.endpoint"); } + + 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"); @@ -395,6 +407,10 @@ public TimeZone getTimeZone() { public boolean getBoolean(String name, boolean defaultValue) { return mProperties.getBoolean(name, defaultValue); } + + public boolean getBoolean(String name) { + return mProperties.getBoolean(name); + } private void checkProperty(String name) { if (!mProperties.containsKey(name)) { diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index c6b09fd25..689204ba6 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -21,7 +21,7 @@ import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.pinterest.secor.common.*; - +import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.regions.Region; import com.amazonaws.regions.Regions; @@ -71,10 +71,22 @@ public S3UploadManager(SecorConfig config) { String region = mConfig.getAwsRegion(); AmazonS3 client; + 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()) { - client = new AmazonS3Client(); + client = new AmazonS3Client(clientConfiguration); } else { - client = new AmazonS3Client(new BasicAWSCredentials(accessKey, secretKey)); + client = new AmazonS3Client(new BasicAWSCredentials(accessKey, secretKey),clientConfiguration); } if (!endpoint.isEmpty()) { From 41566dcca1a44bf2704d42248d882a141274cb70 Mon Sep 17 00:00:00 2001 From: treff7es Date: Fri, 12 Feb 2016 15:08:27 +0100 Subject: [PATCH 140/330] Adding compression codec class to CompressionCodecFactory's confuguration otherwise ccf.getCodecByClassName won't find the set codec. --- .../java/com/pinterest/secor/util/CompressionUtil.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/util/CompressionUtil.java b/src/main/java/com/pinterest/secor/util/CompressionUtil.java index c7d4bf708..b988071f3 100644 --- a/src/main/java/com/pinterest/secor/util/CompressionUtil.java +++ b/src/main/java/com/pinterest/secor/util/CompressionUtil.java @@ -16,6 +16,9 @@ */ 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; @@ -31,8 +34,10 @@ 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( - new Configuration()); + configuration); return ccf.getCodecByClassName(className); } } From e9f72cf95b0482c9ac057e2a6efc988ef9ab7f32 Mon Sep 17 00:00:00 2001 From: Christian Nguyen Van Than Date: Mon, 15 Feb 2016 17:22:23 +0100 Subject: [PATCH 141/330] fix file extension --- .../java/com/pinterest/secor/parser/PartitionFinalizer.java | 2 +- src/main/java/com/pinterest/secor/writer/MessageWriter.java | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index c9bbebc8e..22bc19e0f 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -54,7 +54,7 @@ public PartitionFinalizer(SecorConfig config) throws Exception { mMessageParser = (TimestampedMessageParser) ReflectionUtil.createMessageParser( mConfig.getMessageParserClass(), mConfig); mQuboleClient = new QuboleClient(mConfig); - if (mConfig.getFileExtension() != null) { + if (mConfig.getFileExtension() != null && !mConfig.getFileExtension().isEmpty()) { mFileExtension = mConfig.getFileExtension(); } else if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { CompressionCodec codec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index d98424a93..a965b6282 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -55,7 +55,9 @@ public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, mConfig = config; mOffsetTracker = offsetTracker; mFileRegistry = fileRegistry; - if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + if (mConfig.getFileExtension() != null && !mConfig.getFileExtension().isEmpty()) { + mFileExtension = mConfig.getFileExtension(); + } else if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { mCodec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); mFileExtension = mCodec.getDefaultExtension(); } else { From 6b9e9361986fc78dfd40f3264c87994d979246ee Mon Sep 17 00:00:00 2001 From: PoTa Date: Fri, 19 Feb 2016 11:07:22 +0100 Subject: [PATCH 142/330] Add direct upload config flag for Google cloud storage, fixes #177. --- src/main/config/secor.common.properties | 7 +++++++ src/main/config/secor.dev.gs.properties | 9 ++++++++- .../java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../com/pinterest/secor/uploader/GsUploadManager.java | 5 +++++ 4 files changed, 24 insertions(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 5034021a6..ba637f03f 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -104,6 +104,13 @@ 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 + # Zookeeper config. zookeeper.session.timeout.ms=3000 zookeeper.sync.time.ms=200 diff --git a/src/main/config/secor.dev.gs.properties b/src/main/config/secor.dev.gs.properties index d914462eb..5eb765bb1 100644 --- a/src/main/config/secor.dev.gs.properties +++ b/src/main/config/secor.dev.gs.properties @@ -19,4 +19,11 @@ secor.gs.path=data # Application credentials configuration file # https://developers.google.com/identity/protocols/application-default-credentials -secor.gs.credentials.path=google_app_credentials.json \ No newline at end of file +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/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 64af04ceb..e8f20c708 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -395,6 +395,10 @@ 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"); } diff --git a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java index 4fe4894c2..e5a762d58 100644 --- a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java @@ -69,6 +69,7 @@ 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); @@ -81,6 +82,10 @@ 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 { From 44b7ca19c5006bf296161000e9de0221a6a5d68d Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Fri, 4 Mar 2016 13:22:24 +0100 Subject: [PATCH 143/330] Add Iso8601MessageParser Enables proper Iso8601 parsing using javax.xml.bind.DatatypeConverter for Java 7 support. --- .../secor/parser/Iso8601MessageParser.java | 69 ++++++++++++++ .../secor/parser/Iso8601ParserTest.java | 89 +++++++++++++++++++ 2 files changed, 158 insertions(+) create mode 100644 src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java create mode 100644 src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java 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..213086b8d --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java @@ -0,0 +1,69 @@ +/** + * 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.text.SimpleDateFormat; +import java.util.Date; + +import net.minidev.json.JSONObject; +import net.minidev.json.JSONValue; +import javax.xml.bind.DatatypeConverter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; + +/** + * Iso8601MessageParser extracts timestamp field (specified by 'message.timestamp.name') + * + * @author Jurriaan Pruis (email@jurriaanpruis.nl) + * + */ +public class Iso8601MessageParser 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"; + + public Iso8601MessageParser(SecorConfig config) { + super(config); + } + + @Override + public String[] extractPartitions(Message message) { + JSONObject jsonObject = (JSONObject) JSONValue.parse(message.getPayload()); + String result[] = { defaultDate }; + + if (jsonObject != null) { + Object fieldValue = getJsonFieldValue(jsonObject); + if (fieldValue != null) { + try { + Date dateFormat = DatatypeConverter.parseDateTime(fieldValue.toString()).getTime(); + SimpleDateFormat outputFormatter = new SimpleDateFormat(defaultFormatter); + result[0] = "dt=" + outputFormatter.format(dateFormat); + } catch (Exception e) { + LOG.warn("Impossible to convert date = {} as ISO-8601. Using date default = {}", + fieldValue.toString(), result[0]); + } + } + } + + return result; + } + +} 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..21214d632 --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java @@ -0,0 +1,89 @@ +/** + * 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 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; + +@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 mISOFormat; + private Message mNestedISOFormat; + private OngoingStubbing getTimestamp; + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + 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); + + 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); + + 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); + + 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); + + 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); + + 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); + } + + @Test + public void testExtractDate() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + + assertEquals("dt=2014-07-30", new Iso8601MessageParser(mConfig).extractPartitions(mFormat1)[0]); + assertEquals("dt=2014-07-29", new Iso8601MessageParser(mConfig).extractPartitions(mFormat2)[0]); + assertEquals("dt=2001-07-04", new Iso8601MessageParser(mConfig).extractPartitions(mFormat3)[0]); + assertEquals("dt=2016-03-02", new Iso8601MessageParser(mConfig).extractPartitions(mFormat4)[0]); + assertEquals("dt=1970-01-01", new Iso8601MessageParser(mConfig).extractPartitions(mInvalidDate)[0]); + } + + @Test + public void testNestedField() throws Exception { + Mockito.when(mConfig.getMessageTimestampNameSeparator()).thenReturn("."); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("meta_data.created"); + + assertEquals("dt=2016-01-11", new Iso8601MessageParser(mConfig).extractPartitions(mNestedISOFormat)[0]); + } +} From be16d8a732f1ffa55ad4a7b0b3da77db1e008af2 Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Sun, 6 Mar 2016 15:34:10 +0100 Subject: [PATCH 144/330] Emit warning on missing Iso8601 field --- .../com/pinterest/secor/parser/Iso8601MessageParser.java | 8 +++++--- .../com/pinterest/secor/parser/Iso8601ParserTest.java | 6 ++++++ 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java index 213086b8d..d41aea83a 100644 --- a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java @@ -39,22 +39,24 @@ public class Iso8601MessageParser 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 static final SimpleDateFormat outputFormatter = new SimpleDateFormat(defaultFormatter); public Iso8601MessageParser(SecorConfig config) { super(config); } @Override - public String[] extractPartitions(Message message) { + public String[] extractPartitions(Message message) throws Exception { JSONObject jsonObject = (JSONObject) JSONValue.parse(message.getPayload()); String result[] = { defaultDate }; if (jsonObject != null) { Object fieldValue = getJsonFieldValue(jsonObject); - if (fieldValue != null) { + if (fieldValue == null) { + LOG.warn("Missing field value. Using default partition = {}", defaultDate); + } else { try { Date dateFormat = DatatypeConverter.parseDateTime(fieldValue.toString()).getTime(); - SimpleDateFormat outputFormatter = new SimpleDateFormat(defaultFormatter); result[0] = "dt=" + outputFormatter.format(dateFormat); } catch (Exception e) { LOG.warn("Impossible to convert date = {} as ISO-8601. Using date default = {}", diff --git a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java index 21214d632..25f719043 100644 --- a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java @@ -38,6 +38,7 @@ public class Iso8601ParserTest extends TestCase { private Message mInvalidDate; private Message mISOFormat; private Message mNestedISOFormat; + private Message mMissingDate; private OngoingStubbing getTimestamp; @Override @@ -66,6 +67,10 @@ public void setUp() throws Exception { 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); + + 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); } @Test @@ -77,6 +82,7 @@ public void testExtractDate() throws Exception { assertEquals("dt=2001-07-04", new Iso8601MessageParser(mConfig).extractPartitions(mFormat3)[0]); assertEquals("dt=2016-03-02", new Iso8601MessageParser(mConfig).extractPartitions(mFormat4)[0]); assertEquals("dt=1970-01-01", new Iso8601MessageParser(mConfig).extractPartitions(mInvalidDate)[0]); + assertEquals("dt=1970-01-01", new Iso8601MessageParser(mConfig).extractPartitions(mMissingDate)[0]); } @Test From 2d2fe5d4a7741804f16364ad312e0b0b4dfa9f57 Mon Sep 17 00:00:00 2001 From: Derek Argueta Date: Mon, 14 Mar 2016 15:34:39 -0600 Subject: [PATCH 145/330] DESIGN.md typo fix --- DESIGN.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/DESIGN.md b/DESIGN.md index 6c0b13ca6..dd95bdbf9 100644 --- a/DESIGN.md +++ b/DESIGN.md @@ -24,7 +24,7 @@ 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 the most recent [Kafka] version (0.8) which offers significant improvements over 0.7, and it comes with Go language bindings (required by other pieces of the Ads infra). No-goals: From 2354b77e7e3ecceebb5f5149f2b7e3de8a67e225 Mon Sep 17 00:00:00 2001 From: Jorge Montero Date: Tue, 15 Mar 2016 11:24:44 -0700 Subject: [PATCH 146/330] make sure we print the offset in the log statement --- src/main/java/com/pinterest/secor/common/KafkaClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index de6198633..6da05cc69 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -113,7 +113,7 @@ private long findLastOffset(TopicPartition topicPartition, SimpleConsumer consum private Message getMessage(TopicPartition topicPartition, long offset, SimpleConsumer consumer) { - LOG.debug("fetching message topic {} partition {} offset ", + 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); From 83199b3b472cd3ccb852c58628dd0c8392a241e5 Mon Sep 17 00:00:00 2001 From: Lovenish Goyal Date: Sat, 19 Mar 2016 10:59:16 +0530 Subject: [PATCH 147/330] code changes according reviews --- .../parser/TimestampedMessageParser.java | 90 +++++++--- .../secor/parser/JsonMessageParserTest.java | 162 +++++++++++++++++- 2 files changed, 226 insertions(+), 26 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index e61a11d5e..2b7d4b8c3 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. @@ -31,6 +31,7 @@ public abstract class TimestampedMessageParser extends MessageParser implements 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 @@ -41,13 +42,19 @@ public abstract class TimestampedMessageParser extends MessageParser implements private final SimpleDateFormat mHrFormatter; private final SimpleDateFormat mDtHrFormatter; private final int mFinalizerDelaySeconds; + private final SimpleDateFormat mDtHrMinFormatter; + private final SimpleDateFormat mMinFormatter; private final boolean mUsingHourly; + private final boolean mUsingMinutely; + public TimestampedMessageParser(SecorConfig config) { super(config); mUsingHourly = usingHourly(config); + mUsingMinutely = usingMinutely(config); LOG.info("UsingHourly: {}", mUsingHourly); + LOG.info("UsingMin: {}", mUsingMinutely); mFinalizerDelaySeconds = config.getFinalizerDelaySeconds(); LOG.info("FinalizerDelaySeconds: {}", mFinalizerDelaySeconds); @@ -57,14 +64,20 @@ public TimestampedMessageParser(SecorConfig config) { mHrFormatter.setTimeZone(config.getTimeZone()); mDtHrFormatter = new SimpleDateFormat("yyyy-MM-dd-HH"); mDtHrFormatter.setTimeZone(config.getTimeZone()); + mDtHrMinFormatter = new SimpleDateFormat("yyyy-MM-dd-HH-mm"); + mDtHrMinFormatter.setTimeZone(config.getTimeZone()); + mMinFormatter = new SimpleDateFormat("mm"); + mMinFormatter.setTimeZone(config.getTimeZone()); } - public abstract long extractTimestampMillis(final Message message) throws Exception; - static boolean usingHourly(SecorConfig config) { return config.getBoolean("partitioner.granularity.hour", false); } + static boolean usingMinutely(SecorConfig config) { + return config.getBoolean("partitioner.granularity.minute", false); + } + protected static long toMillis(final long timestamp) { final long nanosecondDivider = (long) Math.pow(10, 9 + 9); final long millisecondDivider = (long) Math.pow(10, 9 + 3); @@ -79,12 +92,17 @@ protected static long toMillis(final long timestamp) { return timestampMillis; } - protected String[] generatePartitions(long timestampMillis, boolean usingHourly) - throws Exception { + public abstract long extractTimestampMillis(final Message message) throws Exception; + + protected String[] generatePartitions(long timestampMillis, boolean usingHourly, boolean usingMinutely) + throws Exception { Date date = new Date(timestampMillis); String dt = "dt=" + mDtFormatter.format(date); String hr = "hr=" + mHrFormatter.format(date); - if (usingHourly) { + String min = "min=" + mMinFormatter.format(date); + if (usingMinutely) { + return new String[]{dt, hr, min}; + } else if (usingHourly) { return new String[]{dt, hr}; } else { return new String[]{dt}; @@ -94,8 +112,9 @@ protected String[] generatePartitions(long timestampMillis, boolean usingHourly) protected long parsePartitions(String[] partitions) throws Exception { String dtValue = partitions[0].split("=")[1]; String hrValue = partitions.length > 1 ? partitions[1].split("=")[1] : "00"; - String value = dtValue + "-" + hrValue; - Date date = mDtHrFormatter.parse(value); + String minValue = partitions.length > 2 ? partitions[2].split("=")[1] : "00"; + String value = dtValue + "-" + hrValue + "-" + minValue; + Date date = mDtHrMinFormatter.parse(value); return date.getTime(); } @@ -103,7 +122,7 @@ protected long parsePartitions(String[] partitions) throws Exception { public String[] extractPartitions(Message message) throws Exception { // Date constructor takes milliseconds since epoch. long timestampMillis = extractTimestampMillis(message); - return generatePartitions(timestampMillis, mUsingHourly); + return generatePartitions(timestampMillis, mUsingHourly, mUsingMinutely); } private long getFinalizedTimestampMillis(Message lastMessage, @@ -112,7 +131,7 @@ private long getFinalizedTimestampMillis(Message lastMessage, long committedTimestamp = extractTimestampMillis(committedMessage); long now = System.currentTimeMillis(); if (lastTimestamp == committedTimestamp && - (now - lastTimestamp) > mFinalizerDelaySeconds * 1000) { + (now - lastTimestamp) > mFinalizerDelaySeconds * 1000) { LOG.info("No new message coming, use the current time: " + now); return now; } @@ -122,39 +141,58 @@ private long getFinalizedTimestampMillis(Message lastMessage, @Override public String[] getFinalizedUptoPartitions(List lastMessages, List committedMessages) throws Exception { + if (lastMessages == null || committedMessages == null) { - LOG.error("Either: {} and {} is null", lastMessages, - committedMessages); + 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)); + 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); + 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); - } + return generatePartitions(minMillis, mUsingHourly, mUsingMinutely); + } @Override public String[] getPreviousPartitions(String[] partitions) throws Exception { long millis = parsePartitions(partitions); boolean usingHourly = mUsingHourly; - if (mUsingHourly && millis % DAY_IN_MILLIS == 0) { + 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 currrent 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] @@ -163,7 +201,7 @@ public String[] getPreviousPartitions(String[] partitions) throws Exception { // dt=07-07, hr=00 // dt=07-06 // dt=07-06, hr=23 - if (partitions.length == 2 ) { + if (partitions.length == 2) { usingHourly = false; millis -= DAY_IN_MILLIS; } else { @@ -173,9 +211,11 @@ public String[] getPreviousPartitions(String[] partitions) throws Exception { } } else { long delta = mUsingHourly ? HOUR_IN_MILLIS : DAY_IN_MILLIS; + if (mUsingMinutely) { + delta = MINUTE_IN_MILLIS; + } millis -= delta; } - return generatePartitions(millis, usingHourly); + return generatePartitions(millis, usingHourly, usingMinutely); + } } - -} diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index 25bcbe17f..39c6c88b7 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -255,5 +255,165 @@ public void testHourlyGetFinalizedUptoPartitions() throws Exception { 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); + } + } } From 258688c7a8cb02c77b7d7f6f8bf4beae5ba21b47 Mon Sep 17 00:00:00 2001 From: Lovenish Goyal Date: Sat, 19 Mar 2016 11:32:01 +0530 Subject: [PATCH 148/330] config --- src/main/config/secor.common.properties | 2 +- .../com/pinterest/secor/parser/TimestampedMessageParser.java | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index ba637f03f..6b59f66cb 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -193,7 +193,7 @@ secor.topic_partition.forget.seconds=600 # s3n://.../topic/dt=2015-07-07/hr=02 # The hour folder ranges from 00 to 23 # partitioner.granularity.hour=true - +# partitioner.granularity.minute=true # how many seconds should the finalizer wait to finalize a partition partitioner.finalizer.delay.seconds=3600 diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 2b7d4b8c3..abedf628e 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -60,10 +60,13 @@ public TimestampedMessageParser(SecorConfig config) { mDtFormatter = new SimpleDateFormat("yyyy-MM-dd"); mDtFormatter.setTimeZone(config.getTimeZone()); + mHrFormatter = new SimpleDateFormat("HH"); mHrFormatter.setTimeZone(config.getTimeZone()); + mDtHrFormatter = new SimpleDateFormat("yyyy-MM-dd-HH"); mDtHrFormatter.setTimeZone(config.getTimeZone()); + mDtHrMinFormatter = new SimpleDateFormat("yyyy-MM-dd-HH-mm"); mDtHrMinFormatter.setTimeZone(config.getTimeZone()); mMinFormatter = new SimpleDateFormat("mm"); From 604c4972e116a3442c1f24ca86b9371f6b5fb5e0 Mon Sep 17 00:00:00 2001 From: Lovenish Goyal Date: Tue, 22 Mar 2016 00:22:06 +0530 Subject: [PATCH 149/330] make congif method public & use timezone from config --- .../pinterest/secor/common/SecorConfig.java | 12 +++++------ .../secor/parser/DateMessageParser.java | 20 ++++++++++++------- .../secor/parser/Iso8601MessageParser.java | 1 + .../secor/parser/DateMessageParserTest.java | 3 +++ .../secor/parser/Iso8601ParserTest.java | 5 ++++- 5 files changed, 27 insertions(+), 14 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index e8f20c708..efc133bcf 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -416,32 +416,32 @@ public boolean getBoolean(String name) { return mProperties.getBoolean(name); } - private void checkProperty(String 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 int getInt(String name) { checkProperty(name); return mProperties.getInt(name); } - private int getInt(String name, int defaultValue) { + public int getInt(String name, int defaultValue) { return mProperties.getInt(name, defaultValue); } - private long getLong(String name) { + public long getLong(String name) { return mProperties.getLong(name); } - private String[] getStringArray(String name) { + public String[] getStringArray(String name) { return mProperties.getStringArray(name); } } diff --git a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java index 733483ba2..c09361c9b 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,6 +26,9 @@ 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') * and the date pattern (specified by 'message.timestamp.input.pattern') @@ -41,9 +42,17 @@ 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"; - + SimpleDateFormat outputFormatter = new SimpleDateFormat(defaultFormatter); + Object inputPattern; + SimpleDateFormat inputFormatter; + public DateMessageParser(SecorConfig config) { super(config); + TimeZone timeZone = config.getTimeZone(); + inputPattern = mConfig.getMessageTimestampInputPattern(); + inputFormatter = new SimpleDateFormat(inputPattern.toString()); + inputFormatter.setTimeZone(timeZone); + outputFormatter.setTimeZone(timeZone); } @Override @@ -53,11 +62,8 @@ public String[] extractPartitions(Message message) { if (jsonObject != null) { Object fieldValue = getJsonFieldValue(jsonObject); - Object inputPattern = mConfig.getMessageTimestampInputPattern(); 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; diff --git a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java index d41aea83a..2e187b53c 100644 --- a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java @@ -43,6 +43,7 @@ public class Iso8601MessageParser extends MessageParser { public Iso8601MessageParser(SecorConfig config) { super(config); + outputFormatter.setTimeZone(config.getTimeZone()); } @Override diff --git a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index db5dfabfe..7db0ed92a 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -18,6 +18,8 @@ import junit.framework.TestCase; +import java.util.TimeZone; + import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mockito; @@ -42,6 +44,7 @@ public class DateMessageParserTest extends TestCase { @Override public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); 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"); diff --git a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java index 25f719043..83eb57437 100644 --- a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java @@ -18,6 +18,8 @@ import junit.framework.TestCase; +import java.util.TimeZone; + import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mockito; @@ -44,6 +46,8 @@ public class Iso8601ParserTest extends TestCase { @Override public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); + 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); @@ -76,7 +80,6 @@ public void setUp() throws Exception { @Test public void testExtractDate() throws Exception { Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); - assertEquals("dt=2014-07-30", new Iso8601MessageParser(mConfig).extractPartitions(mFormat1)[0]); assertEquals("dt=2014-07-29", new Iso8601MessageParser(mConfig).extractPartitions(mFormat2)[0]); assertEquals("dt=2001-07-04", new Iso8601MessageParser(mConfig).extractPartitions(mFormat3)[0]); From e87af757c187e9b292a3ab0c1d94de5ec622faaa Mon Sep 17 00:00:00 2001 From: Lovenish Goyal Date: Wed, 23 Mar 2016 13:52:44 +0530 Subject: [PATCH 150/330] changes as per review comment --- .../java/com/pinterest/secor/parser/DateMessageParser.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java index c09361c9b..74067994f 100644 --- a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java @@ -42,9 +42,9 @@ 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"; - SimpleDateFormat outputFormatter = new SimpleDateFormat(defaultFormatter); - Object inputPattern; - SimpleDateFormat inputFormatter; + protected SimpleDateFormat outputFormatter = new SimpleDateFormat(defaultFormatter); + protected Object inputPattern; + protected SimpleDateFormat inputFormatter; public DateMessageParser(SecorConfig config) { super(config); From aa295f0577db25f8f297feed60401f3d832fb452 Mon Sep 17 00:00:00 2001 From: tcnksm Date: Wed, 23 Mar 2016 14:49:18 +0900 Subject: [PATCH 151/330] Allow to use MS Azure blob storage --- README.md | 2 + pom.xml | 5 ++ src/main/config/secor.common.properties | 2 +- src/main/config/secor.dev.azure.properties | 28 +++++++ .../pinterest/secor/common/SecorConfig.java | 10 +++ .../secor/uploader/AzureUploadManager.java | 78 +++++++++++++++++++ .../com/pinterest/secor/util/FileUtil.java | 2 + 7 files changed, 126 insertions(+), 1 deletion(-) create mode 100644 src/main/config/secor.dev.azure.properties create mode 100644 src/main/java/com/pinterest/secor/uploader/AzureUploadManager.java diff --git a/README.md b/README.md index 6af9807a1..cddebc00a 100644 --- a/README.md +++ b/README.md @@ -121,6 +121,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Zack Dever](https://github.com/zackdever) * [Leo Woessner](https://github.com/estezz) * [Jerome Gagnon](https://github.com/jgagnon1) + * [Taichi Nakashima](https://github.com/tcnksm) ## Companies who use Secor @@ -133,6 +134,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Skyscanner](http://www.skyscanner.net) * [Nextperf](http://www.nextperf.com) * [Zalando](http://www.zalando.com) + * [Rakuten](http://techblog.rakuten.co.jp/) ## Help diff --git a/pom.xml b/pom.xml index 232be028c..729b9c458 100644 --- a/pom.xml +++ b/pom.xml @@ -269,6 +269,11 @@ + + com.microsoft.azure + azure-storage + 4.0.0 + diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 6b59f66cb..bd9df61b3 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -21,7 +21,7 @@ secor.kafka.topic_filter=.* # Choose what to fill according to the service you are using -# in the choice option you can fill S3, GS or Swift +# in the choice option you can fill S3, GS, Swift or Azure cloud.service=S3 # AWS authentication credentials. diff --git a/src/main/config/secor.dev.azure.properties b/src/main/config/secor.dev.azure.properties new file mode 100644 index 000000000..871f4eeec --- /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 sotrage 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/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index e8f20c708..8206e88ba 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -403,6 +403,16 @@ public int getFinalizerDelaySeconds() { return getInt("partitioner.finalizer.delay.seconds"); } + 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 TimeZone getTimeZone() { String timezone = getString("secor.parser.timezone"); return Strings.isNullOrEmpty(timezone) ? TimeZone.getTimeZone("UTC") : TimeZone.getTimeZone(timezone); 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/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index d46343c5d..c7274a570 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -90,6 +90,8 @@ public static String getPrefix(String topic, SecorConfig config) throws IOExcep 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; } From 8f6cbccd9db23d079e11d8750e9830c636f4bd54 Mon Sep 17 00:00:00 2001 From: Yejun Yang Date: Wed, 6 Apr 2016 16:10:58 -0700 Subject: [PATCH 152/330] add docker test --- .gitignore | 1 + Makefile | 12 ++++++++++++ containers/trusty/Dockerfile | 9 +++++++++ containers/xenial/Dockerfile | 9 +++++++++ 4 files changed, 31 insertions(+) create mode 100644 containers/trusty/Dockerfile create mode 100644 containers/xenial/Dockerfile diff --git a/.gitignore b/.gitignore index f1e123720..290c183aa 100644 --- a/.gitignore +++ b/.gitignore @@ -43,3 +43,4 @@ bin/ .settings/ .project .classpath +.m2 diff --git a/Makefile b/Makefile index 90cda6b02..73de33174 100644 --- a/Makefile +++ b/Makefile @@ -3,6 +3,7 @@ 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) @@ -21,3 +22,14 @@ integration: build 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/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 From ee0ed52fb97e3d048369ffccd9d6c53bb4ca6a64 Mon Sep 17 00:00:00 2001 From: Yejun Yang Date: Fri, 8 Apr 2016 17:01:55 -0700 Subject: [PATCH 153/330] add Aws Role support --- pom.xml | 7 ++++- src/main/config/secor.common.properties | 1 + .../pinterest/secor/common/SecorConfig.java | 4 +++ .../secor/uploader/S3UploadManager.java | 29 +++++++++++++++---- 4 files changed, 34 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index 729b9c458..a5c6bcd03 100644 --- a/pom.xml +++ b/pom.xml @@ -94,7 +94,12 @@ com.amazonaws aws-java-sdk-s3 - 1.10.39 + 1.10.68 + + + com.amazonaws + aws-java-sdk-sts + 1.10.68 net.java.dev.jets3t diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index bd9df61b3..a223767fc 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -28,6 +28,7 @@ cloud.service=S3 # 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 97b492c76..5e6af808b 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -237,6 +237,10 @@ public String getAwsSecretKey() { public String getAwsEndpoint() { return getString("aws.endpoint"); } + + public String getAwsRole() { + return getString("aws.role"); + } public boolean getAwsProxyEnabled(){ return getBoolean("aws.proxy.isEnabled"); diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index 689204ba6..f25432a9b 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -29,6 +29,10 @@ import com.amazonaws.services.s3.AmazonS3Client; 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; @@ -65,11 +69,13 @@ public class S3UploadManager extends UploadManager { public S3UploadManager(SecorConfig config) { super(config); - String accessKey = mConfig.getAwsAccessKey(); - String secretKey = mConfig.getAwsSecretKey(); - String endpoint = mConfig.getAwsEndpoint(); - String region = mConfig.getAwsRegion(); + 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(); AmazonS3 client; + AWSCredentialsProvider provider; ClientConfiguration clientConfiguration = new ClientConfiguration(); boolean isHttpProxyEnabled = mConfig.getAwsProxyEnabled(); @@ -84,11 +90,22 @@ public S3UploadManager(SecorConfig config) { } if (accessKey.isEmpty() || secretKey.isEmpty()) { - client = new AmazonS3Client(clientConfiguration); + provider = new DefaultAWSCredentialsProviderChain(); } else { - client = new AmazonS3Client(new BasicAWSCredentials(accessKey, secretKey),clientConfiguration); + 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 (!endpoint.isEmpty()) { client.setEndpoint(endpoint); } else if (!region.isEmpty()) { From 073ac76b4957246f5f7a9f8dc8d615c522e011fd Mon Sep 17 00:00:00 2001 From: Mathias Ehlert Date: Mon, 18 Apr 2016 09:41:16 +0200 Subject: [PATCH 154/330] Reorder handling of codec and file-extension In the current state, if the config defines both compression codec and file extension, the compression codec is not applied. Setting a compression codec should not rely on file extension to NOT be set. Especially since the comments in the config file state "gz" as an example for `secor.file.extension` --- .../java/com/pinterest/secor/writer/MessageWriter.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index a965b6282..31a3fe50a 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -55,14 +55,16 @@ public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, mConfig = config; mOffsetTracker = offsetTracker; mFileRegistry = fileRegistry; - if (mConfig.getFileExtension() != null && !mConfig.getFileExtension().isEmpty()) { - mFileExtension = mConfig.getFileExtension(); - } else if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { 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(); } From d2f6a49838c29fe372b72f3c2d3f1a967b1737df Mon Sep 17 00:00:00 2001 From: pritamm Date: Mon, 18 Apr 2016 15:03:15 +0530 Subject: [PATCH 155/330] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index cddebc00a..3bf89d110 100644 --- a/README.md +++ b/README.md @@ -4,7 +4,7 @@ Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud Storage] and [Openstack Swift]. -## Key features +## 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, From 3425d847d94d1ad058109e60717ee0127b71b1cb Mon Sep 17 00:00:00 2001 From: Pritam Metha Date: Mon, 18 Apr 2016 15:14:21 +0530 Subject: [PATCH 156/330] Add hashing method to use in the S3 path prefix --- .../com/pinterest/secor/util/FileUtil.java | 60 ++++++++++++++++--- 1 file changed, 52 insertions(+), 8 deletions(-) diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index c7274a570..46dbe8328 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -16,26 +16,36 @@ */ package com.pinterest.secor.util; -import com.pinterest.secor.common.SecorConfig; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.s3a.Constants; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - 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 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 com.pinterest.secor.common.SecorConfig; + /** * File util implements utilities for interactions with the file system. * * @author Pawel Garbacki (pawel@pinterest.com) */ public class FileUtil { + private static final Logger LOG = LoggerFactory.getLogger(FileUtil.class); private static 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'}; public static void configure(SecorConfig config) { if (config != null) { @@ -96,7 +106,6 @@ public static String getPrefix(String topic, SecorConfig config) throws IOExcep return prefix; } - public static String[] list(String path) throws IOException { FileSystem fs = getFileSystem(path); Path fsPath = new Path(path); @@ -188,4 +197,39 @@ public static long getModificationTimeMsRecursive(String path) throws IOExceptio } return modificationTime; } + + /** Generat MD5 hash of topic and paritions. 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); + } } From 5d9c6e58ff01ea0bc5ecbdd1fcd835a1c73f651b Mon Sep 17 00:00:00 2001 From: Pritam Metha Date: Mon, 18 Apr 2016 15:25:27 +0530 Subject: [PATCH 157/330] Add md5hash to the s3 path prefix for better paritioning of the data on s3 --- .../java/com/pinterest/secor/uploader/S3UploadManager.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index f25432a9b..d0e8e5fa8 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -21,6 +21,7 @@ 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; @@ -117,7 +118,9 @@ public void refresh() {} public Handle upload(LogFilePath localPath) throws Exception { String s3Bucket = mConfig.getS3Bucket(); - String s3Key = localPath.withPrefix(mConfig.getS3Path()).getLogFilePath(); + // add MD5 hash to the prefix to have proper partitioning of the secor logs on s3 + String md5Hash = FileUtil.getMd5Hash(localPath.getTopic(), localPath.getPartitions()); + String s3Key = localPath.withPrefix(md5Hash + "/" + mConfig.getS3Path()).getLogFilePath(); File localFile = new File(localPath.getLogFilePath()); // make upload request, taking into account configured options for encryption From 6597988c84516768b587c297f43eff6d7834c7a5 Mon Sep 17 00:00:00 2001 From: Pritam Metha Date: Tue, 19 Apr 2016 12:08:51 +0530 Subject: [PATCH 158/330] #199 Add parameter to control prefixing md5hash to the s3 path for better partitioning of data --- src/main/config/secor.common.properties | 4 ++ .../pinterest/secor/common/SecorConfig.java | 13 ++++--- .../secor/uploader/S3UploadManager.java | 38 +++++++------------ 3 files changed, 25 insertions(+), 30 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index a223767fc..cf501adb3 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -291,3 +291,7 @@ 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 \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 5e6af808b..32d468667 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -16,15 +16,14 @@ */ 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.Map; import java.util.Properties; import java.util.TimeZone; +import javax.naming.ConfigurationException; + +import com.sun.xml.internal.ws.util.StringUtils; + /** * One-stop shop for Secor configuration options. * @@ -407,6 +406,10 @@ 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"); } diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index d0e8e5fa8..08aa79516 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -16,30 +16,12 @@ */ 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.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 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. @@ -118,9 +100,15 @@ public void refresh() {} public Handle upload(LogFilePath localPath) throws Exception { String s3Bucket = mConfig.getS3Bucket(); - // add MD5 hash to the prefix to have proper partitioning of the secor logs on s3 - String md5Hash = FileUtil.getMd5Hash(localPath.getTopic(), localPath.getPartitions()); - String s3Key = localPath.withPrefix(md5Hash + "/" + mConfig.getS3Path()).getLogFilePath(); + String s3Key = null; + 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 + "/" + mConfig.getS3Path()).getLogFilePath(); + } + else { + s3Key = localPath.withPrefix(mConfig.getS3Path()).getLogFilePath(); + } File localFile = new File(localPath.getLogFilePath()); // make upload request, taking into account configured options for encryption From 941d412d305d170817e0a1bee74a0595b4a35b13 Mon Sep 17 00:00:00 2001 From: Pritam Metha Date: Mon, 18 Apr 2016 15:25:27 +0530 Subject: [PATCH 159/330] Add md5hash to the s3 path prefix for better paritioning of the data on s3 Update README.md Add hashing method to use in the S3 path prefix --- README.md | 2 +- src/main/config/secor.common.properties | 4 ++ .../pinterest/secor/common/SecorConfig.java | 13 ++-- .../secor/uploader/S3UploadManager.java | 35 ++++------- .../com/pinterest/secor/util/FileUtil.java | 60 ++++++++++++++++--- 5 files changed, 78 insertions(+), 36 deletions(-) diff --git a/README.md b/README.md index cddebc00a..3bf89d110 100644 --- a/README.md +++ b/README.md @@ -4,7 +4,7 @@ Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud Storage] and [Openstack Swift]. -## Key features +## 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, diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index a223767fc..cf501adb3 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -291,3 +291,7 @@ 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 \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 5e6af808b..32d468667 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -16,15 +16,14 @@ */ 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.Map; import java.util.Properties; import java.util.TimeZone; +import javax.naming.ConfigurationException; + +import com.sun.xml.internal.ws.util.StringUtils; + /** * One-stop shop for Secor configuration options. * @@ -407,6 +406,10 @@ 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"); } diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index f25432a9b..08aa79516 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -16,29 +16,12 @@ */ 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.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.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 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. @@ -117,7 +100,15 @@ public void refresh() {} public Handle upload(LogFilePath localPath) throws Exception { String s3Bucket = mConfig.getS3Bucket(); - String s3Key = localPath.withPrefix(mConfig.getS3Path()).getLogFilePath(); + String s3Key = null; + 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 + "/" + mConfig.getS3Path()).getLogFilePath(); + } + else { + s3Key = localPath.withPrefix(mConfig.getS3Path()).getLogFilePath(); + } File localFile = new File(localPath.getLogFilePath()); // make upload request, taking into account configured options for encryption diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index c7274a570..46dbe8328 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -16,26 +16,36 @@ */ package com.pinterest.secor.util; -import com.pinterest.secor.common.SecorConfig; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.s3a.Constants; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - 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 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 com.pinterest.secor.common.SecorConfig; + /** * File util implements utilities for interactions with the file system. * * @author Pawel Garbacki (pawel@pinterest.com) */ public class FileUtil { + private static final Logger LOG = LoggerFactory.getLogger(FileUtil.class); private static 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'}; public static void configure(SecorConfig config) { if (config != null) { @@ -96,7 +106,6 @@ public static String getPrefix(String topic, SecorConfig config) throws IOExcep return prefix; } - public static String[] list(String path) throws IOException { FileSystem fs = getFileSystem(path); Path fsPath = new Path(path); @@ -188,4 +197,39 @@ public static long getModificationTimeMsRecursive(String path) throws IOExceptio } return modificationTime; } + + /** Generat MD5 hash of topic and paritions. 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); + } } From 9ca59678fa16bc206e7fec3f5a4a38b2b460def7 Mon Sep 17 00:00:00 2001 From: Pritam Metha Date: Tue, 19 Apr 2016 12:21:50 +0530 Subject: [PATCH 160/330] #199 Add missing imports --- .../pinterest/secor/common/SecorConfig.java | 7 ++++-- .../secor/uploader/S3UploadManager.java | 22 +++++++++++++++++++ 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 32d468667..c9349d5e9 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -16,13 +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.Map; import java.util.Properties; import java.util.TimeZone; -import javax.naming.ConfigurationException; -import com.sun.xml.internal.ws.util.StringUtils; /** * One-stop shop for Secor configuration options. diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index 08aa79516..fcf8d77f8 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -16,6 +16,28 @@ */ 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.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 com.pinterest.secor.common.LogFilePath; From fc883a1e85e74fc5e152b2e0977e88049a559c07 Mon Sep 17 00:00:00 2001 From: Pritam Metha Date: Tue, 19 Apr 2016 12:28:44 +0530 Subject: [PATCH 161/330] #199 Remove unnecessary imports --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index e75147225..666ea672c 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -25,11 +25,6 @@ import java.util.Properties; import java.util.TimeZone; -import javax.naming.ConfigurationException; - -import com.sun.xml.internal.ws.util.StringUtils; - - /** * One-stop shop for Secor configuration options. * From 9cfae06950ff7ea226c3512ff9262628045079c0 Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Tue, 19 Apr 2016 00:10:48 -0700 Subject: [PATCH 162/330] allow uploader to upload files to a new s3 path after a given migration date --- .../pinterest/secor/common/SecorConfig.java | 8 ++++++++ .../secor/uploader/S3UploadManager.java | 20 ++++++++++++++++--- 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 5e6af808b..9529f3c0d 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -171,6 +171,14 @@ public String getS3Path() { return getString("secor.s3.path"); } + public String getS3MigrationPath() { + return getString("secor.s3.migration.path"); + } + + public String getS3MigrationDate() { + return getString("secor.s3.migration.date"); + } + public String getS3Prefix() { return getS3FileSystem() + "://" + getS3Bucket() + "/" + getS3Path(); } diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index f25432a9b..fe13a75a9 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -38,6 +38,8 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.text.SimpleDateFormat; +import java.util.Date; /** * Manages uploads to S3 using the TransferManager class from the AWS @@ -117,11 +119,23 @@ public void refresh() {} public Handle upload(LogFilePath localPath) throws Exception { String s3Bucket = mConfig.getS3Bucket(); - String s3Key = localPath.withPrefix(mConfig.getS3Path()).getLogFilePath(); + Date s3MigrationDate = null; + String s3Path; + try { + s3MigrationDate = new SimpleDateFormat("yyyy-MM-dd").parse(mConfig.getS3MigrationDate()); + } catch (RuntimeException e) { + // optional migration date, so do nothing if it's not defined + } + if (s3MigrationDate != null && !s3MigrationDate.after(new Date())) { + s3Path = mConfig.getS3MigrationPath(); + } + else { + s3Path = mConfig.getS3Path(); + } + String s3Key = localPath.withPrefix(s3Path).getLogFilePath(); File localFile = new File(localPath.getLogFilePath()); - // make upload request, taking into account configured options for encryption - PutObjectRequest uploadRequest = new PutObjectRequest(s3Bucket, s3Key, localFile);; + 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); From 9a2cd505e2b3305c75f63fd6640ad378654b5dcf Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Tue, 19 Apr 2016 23:23:42 -0700 Subject: [PATCH 163/330] allow hadoops3manager to upload to a new s3 path after a given date and changed variable naming conventions --- README.md | 2 +- src/main/config/secor.common.properties | 10 +++ .../pinterest/secor/common/SecorConfig.java | 17 +++- .../secor/uploader/S3UploadManager.java | 45 ++++++++--- .../com/pinterest/secor/util/FileUtil.java | 79 ++++++++++++++++--- 5 files changed, 129 insertions(+), 24 deletions(-) diff --git a/README.md b/README.md index cddebc00a..3bf89d110 100644 --- a/README.md +++ b/README.md @@ -4,7 +4,7 @@ Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud Storage] and [Openstack Swift]. -## Key features +## 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, diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index a223767fc..c558c581c 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -291,3 +291,13 @@ 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= \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 9529f3c0d..0caa712d9 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -171,17 +171,22 @@ public String getS3Path() { return getString("secor.s3.path"); } - public String getS3MigrationPath() { - return getString("secor.s3.migration.path"); + public String getS3AlternativePath() { + return getString("secor.s3.alternative.path"); } - public String getS3MigrationDate() { - return getString("secor.s3.migration.date"); + public String getS3AlterPathDate() { + return getString("secor.s3.alter.path.date"); } public String getS3Prefix() { return getS3FileSystem() + "://" + getS3Bucket() + "/" + getS3Path(); } + + public String getS3AlternativePrefix() { + return getS3FileSystem() + "://" + getS3Bucket() + "/" + getS3AlternativePath(); + } + public String getLocalPath() { return getString("secor.local.path"); } @@ -415,6 +420,10 @@ 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"); } diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index fe13a75a9..d6d7d42ed 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -21,6 +21,7 @@ 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; @@ -41,6 +42,10 @@ 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. @@ -66,7 +71,11 @@ public class S3UploadManager extends UploadManager { private static final String S3 = "S3"; private static final String CUSTOMER = "customer"; + private final String s3Path; + private final String s3AlternativePath; + private TransferManager mManager; + private Date s3AlterPathDate = null; public S3UploadManager(SecorConfig config) { super(config); @@ -76,6 +85,19 @@ public S3UploadManager(SecorConfig config) { final String endpoint = mConfig.getAwsEndpoint(); final String region = mConfig.getAwsRegion(); final String awsRole = mConfig.getAwsRole(); + + s3Path = mConfig.getS3Path(); + s3AlternativePath = mConfig.getS3AlternativePath(); + + final String s3AlterPathDateString = mConfig.getS3AlterPathDate(); + try { + if (s3AlterPathDateString != null && !s3AlterPathDateString.isEmpty()) { + s3AlterPathDate = new SimpleDateFormat("yyyy-MM-dd").parse(s3AlterPathDateString); + } + } catch (Exception e) { + LOG.error(e.getMessage() + " Date format needs to be yyyy-MM-dd."); + } + AmazonS3 client; AWSCredentialsProvider provider; @@ -119,20 +141,23 @@ public void refresh() {} public Handle upload(LogFilePath localPath) throws Exception { String s3Bucket = mConfig.getS3Bucket(); - Date s3MigrationDate = null; - String s3Path; - try { - s3MigrationDate = new SimpleDateFormat("yyyy-MM-dd").parse(mConfig.getS3MigrationDate()); - } catch (RuntimeException e) { - // optional migration date, so do nothing if it's not defined + String curS3Path = null; + if (s3AlterPathDate != null && !s3AlterPathDate.after(new Date())) { + curS3Path = s3AlternativePath; + LOG.info("Will upload file to alternative s3 path s3://{}/{}", s3Bucket, curS3Path); + } + else { + curS3Path = s3Path; } - if (s3MigrationDate != null && !s3MigrationDate.after(new Date())) { - s3Path = mConfig.getS3MigrationPath(); + String s3Key = null; + 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 { - s3Path = mConfig.getS3Path(); + s3Key = localPath.withPrefix(curS3Path).getLogFilePath(); } - String s3Key = localPath.withPrefix(s3Path).getLogFilePath(); File localFile = new File(localPath.getLogFilePath()); // make upload request, taking into account configured options for encryption PutObjectRequest uploadRequest = new PutObjectRequest(s3Bucket, s3Key, localFile); diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index c7274a570..1a2519188 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -16,18 +16,28 @@ */ package com.pinterest.secor.util; -import com.pinterest.secor.common.SecorConfig; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.s3a.Constants; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - 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 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 com.pinterest.secor.common.SecorConfig; /** * File util implements utilities for interactions with the file system. @@ -35,7 +45,10 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class FileUtil { + 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'}; public static void configure(SecorConfig config) { if (config != null) { @@ -87,7 +100,21 @@ public static String getPrefix(String topic, SecorConfig config) throws IOExcep } prefix = "swift://" + container + ".GENERICPROJECT/" + config.getSwiftPath(); } else if (config.getCloudService().equals("S3")) { - prefix = config.getS3Prefix(); + String s3AlterPathDate = config.getS3AlterPathDate(); + if (s3AlterPathDate != null && !s3AlterPathDate.isEmpty()) { + try { + Date alterPathDate = new SimpleDateFormat("yyyy-MM-dd").parse(s3AlterPathDate); + if (alterPathDate != null && !alterPathDate.after(new Date())) { + prefix = config.getS3AlternativePrefix(); + LOG.info("Will upload file to alternative s3 prefix path {}", prefix); + } + } catch (Exception e) { + LOG.error(e.getMessage() + " Date format needs to be yyyy-MM-dd."); + } + } + else { + prefix = config.getS3Prefix(); + } } else if (config.getCloudService().equals("GS")) { prefix = "gs://" + config.getGsBucket() + "/" + config.getGsPath(); } else if (config.getCloudService().equals("Azure")) { @@ -96,7 +123,6 @@ public static String getPrefix(String topic, SecorConfig config) throws IOExcep return prefix; } - public static String[] list(String path) throws IOException { FileSystem fs = getFileSystem(path); Path fsPath = new Path(path); @@ -188,4 +214,39 @@ public static long getModificationTimeMsRecursive(String path) throws IOExceptio } return modificationTime; } + + /** Generat MD5 hash of topic and paritions. 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); + } } From b87ac386e735263a68f095a8f28bda8a7b119d71 Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Tue, 19 Apr 2016 23:37:45 -0700 Subject: [PATCH 164/330] removed unneeded imports; overwritten by last merge --- src/main/config/secor.common.properties | 1 - .../com/pinterest/secor/uploader/S3UploadManager.java | 4 ---- src/main/java/com/pinterest/secor/util/FileUtil.java | 11 ----------- 3 files changed, 16 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index e4b8d8846..a8fc2683b 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -301,4 +301,3 @@ secor.s3.alter.path.date= # An alternative S3 path for secor to upload files to secor.s3.alternative.path= - diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index 7141af428..d6d7d42ed 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -46,10 +46,6 @@ import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.util.FileUtil; -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. diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index fef8cec09..1a2519188 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -39,17 +39,6 @@ import com.pinterest.secor.common.SecorConfig; -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 com.pinterest.secor.common.SecorConfig; - /** * File util implements utilities for interactions with the file system. * From 8ecd2dd863a1f86c4df40705d86da04a9aa6b3a1 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 20 Apr 2016 14:56:14 -0700 Subject: [PATCH 165/330] Bump up the version to 0.17-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a5c6bcd03..28a35f458 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.16-SNAPSHOT + 0.17-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 0788863d8e1d838a05964b6142537ae78005167a Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Mon, 25 Apr 2016 22:37:43 -0700 Subject: [PATCH 166/330] refactored altered s3 path checks to go into FileUtil --- .../secor/uploader/HadoopS3UploadManager.java | 15 +++++-- .../secor/uploader/S3UploadManager.java | 39 ++++++---------- .../com/pinterest/secor/util/FileUtil.java | 44 +++++++++++++------ 3 files changed, 56 insertions(+), 42 deletions(-) diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java index 59be519ff..387d64aa9 100644 --- a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -47,14 +47,23 @@ public Handle upload(LogFilePath localPath) throws Exception { String prefix = FileUtil.getPrefix(localPath.getTopic(), mConfig); LogFilePath path = localPath.withPrefix(prefix); final String localLogFilename = localPath.getLogFilePath(); - final String logFilename = path.getLogFilePath(); - LOG.info("uploading file {} to {}", localLogFilename, logFilename); + final String logFileName; + + if (FileUtil.s3PathPrefixIsAltered(path.getLogFilePath(), mConfig)) { + logFileName = localPath.withPrefix(FileUtil.getS3AlternativePathPrefix(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); + FileUtil.moveToCloud(localLogFilename, logFileName); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index d6d7d42ed..7d8aa7814 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -72,10 +72,8 @@ public class S3UploadManager extends UploadManager { private static final String CUSTOMER = "customer"; private final String s3Path; - private final String s3AlternativePath; private TransferManager mManager; - private Date s3AlterPathDate = null; public S3UploadManager(SecorConfig config) { super(config); @@ -87,16 +85,6 @@ public S3UploadManager(SecorConfig config) { final String awsRole = mConfig.getAwsRole(); s3Path = mConfig.getS3Path(); - s3AlternativePath = mConfig.getS3AlternativePath(); - - final String s3AlterPathDateString = mConfig.getS3AlterPathDate(); - try { - if (s3AlterPathDateString != null && !s3AlterPathDateString.isEmpty()) { - s3AlterPathDate = new SimpleDateFormat("yyyy-MM-dd").parse(s3AlterPathDateString); - } - } catch (Exception e) { - LOG.error(e.getMessage() + " Date format needs to be yyyy-MM-dd."); - } AmazonS3 client; AWSCredentialsProvider provider; @@ -141,24 +129,25 @@ public void refresh() {} public Handle upload(LogFilePath localPath) throws Exception { String s3Bucket = mConfig.getS3Bucket(); - String curS3Path = null; - if (s3AlterPathDate != null && !s3AlterPathDate.after(new Date())) { - curS3Path = s3AlternativePath; - LOG.info("Will upload file to alternative s3 path s3://{}/{}", s3Bucket, curS3Path); - } - else { - curS3Path = s3Path; + 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); } - String s3Key = null; + 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(); + // 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(); + s3Key = localPath.withPrefix(curS3Path).getLogFilePath(); } - File localFile = new File(localPath.getLogFilePath()); + // make upload request, taking into account configured options for encryption PutObjectRequest uploadRequest = new PutObjectRequest(s3Bucket, s3Key, localFile); if (!mConfig.getAwsSseType().isEmpty()) { diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index 1a2519188..ba75abc67 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -84,6 +84,36 @@ public static FileSystem getFileSystem(String path) throws IOException { 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()); + + // s3Key contains the log path, e.g. raw_logs/secor_topic/dt=2016-04-20/3_0_0000000000000292564 + String[] logPathParts = logFileName.split("/"); + for (String part : logPathParts) { + if (part.startsWith("dt=")) { + logDate = new SimpleDateFormat("yyyy-MM-dd").parse(part.replaceFirst("dt=", "")); + break; + } + } + + if (logDate == null) { + LOG.error("Did not find a date in the format yyyy-MM-dd in " + logFileName); + } + if (!s3AlterPathDate.after(logDate)) { + return true; + } + } + + return false; + } + + public static String getS3AlternativePathPrefix(SecorConfig config) { + return config.getS3AlternativePath(); + } + public static String getPrefix(String topic, SecorConfig config) throws IOException { String prefix = null; if (config.getCloudService().equals("Swift")) { @@ -100,21 +130,7 @@ public static String getPrefix(String topic, SecorConfig config) throws IOExcep } prefix = "swift://" + container + ".GENERICPROJECT/" + config.getSwiftPath(); } else if (config.getCloudService().equals("S3")) { - String s3AlterPathDate = config.getS3AlterPathDate(); - if (s3AlterPathDate != null && !s3AlterPathDate.isEmpty()) { - try { - Date alterPathDate = new SimpleDateFormat("yyyy-MM-dd").parse(s3AlterPathDate); - if (alterPathDate != null && !alterPathDate.after(new Date())) { - prefix = config.getS3AlternativePrefix(); - LOG.info("Will upload file to alternative s3 prefix path {}", prefix); - } - } catch (Exception e) { - LOG.error(e.getMessage() + " Date format needs to be yyyy-MM-dd."); - } - } - else { prefix = config.getS3Prefix(); - } } else if (config.getCloudService().equals("GS")) { prefix = "gs://" + config.getGsBucket() + "/" + config.getGsPath(); } else if (config.getCloudService().equals("Azure")) { From 0b298c908ee8a8c4a9aa9043cfaed8870ce82d0f Mon Sep 17 00:00:00 2001 From: yejun Date: Wed, 3 Jun 2015 18:32:18 -0700 Subject: [PATCH 167/330] Use kafka topic blacklist fix import add debug message --- src/main/config/secor.common.properties | 1 + src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../java/com/pinterest/secor/reader/MessageReader.java | 8 +++++++- 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index a8fc2683b..0c79a419e 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -19,6 +19,7 @@ # Regular expression matching names of consumed topics. secor.kafka.topic_filter=.* +secor.kafka.topic_blacklist= # Choose what to fill according to the service you are using # in the choice option you can fill S3, GS, Swift or Azure diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 0caa712d9..d03d92592 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -195,6 +195,10 @@ public String getKafkaTopicFilter() { return getString("secor.kafka.topic_filter"); } + public String getKafkaTopicBlacklist() { + return getString("secor.kafka.topic_blacklist"); + } + public String getKafkaGroup() { return getString("secor.kafka.group"); } diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 2f8ef6d44..b9182ae2e 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -29,6 +29,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; @@ -65,7 +66,12 @@ 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); From d681daf038194cc8ed07d0c6e0ce42383a87ff39 Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Wed, 27 Apr 2016 21:06:23 -0700 Subject: [PATCH 168/330] changed path check to be regex based and partition finalizer to use altered path if needed --- .../secor/parser/PartitionFinalizer.java | 11 ++++++++++ .../com/pinterest/secor/util/FileUtil.java | 20 ++++++++++--------- 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 22bc19e0f..fe8d0fa24 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -101,6 +101,11 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw LOG.info("Looking for partition: " + Arrays.toString(previous)); LogFilePath logFilePath = new LogFilePath(prefix, topic, previous, mConfig.getGeneration(), 0, 0, mFileExtension); + + if (FileUtil.s3PathPrefixIsAltered(logFilePath.getLogFilePath(), mConfig)) { + logFilePath = logFilePath.withPrefix(FileUtil.getS3AlternativePathPrefix(mConfig)); + } + String logFileDir = logFilePath.getLogFileDir(); if (FileUtil.exists(logFileDir)) { String successFilePath = logFileDir + "/_SUCCESS"; @@ -175,6 +180,12 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw // 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.getS3AlternativePathPrefix(mConfig)); + LOG.info("Will finalize alternative s3 logFilePath {}", logFilePath); + } + String logFileDir = logFilePath.getLogFileDir(); String successFilePath = logFileDir + "/_SUCCESS"; diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index ba75abc67..225423576 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -27,6 +27,8 @@ 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; @@ -49,6 +51,7 @@ public class FileUtil { 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) { if (config != null) { @@ -88,20 +91,19 @@ public static boolean s3PathPrefixIsAltered(String logFileName, SecorConfig conf throws Exception { Date logDate = null; if (config.getS3AlterPathDate() != null && !config.getS3AlterPathDate().isEmpty()) { + Date s3AlterPathDate = new SimpleDateFormat("yyyy-MM-dd").parse(config.getS3AlterPathDate()); - // s3Key contains the log path, e.g. raw_logs/secor_topic/dt=2016-04-20/3_0_0000000000000292564 - String[] logPathParts = logFileName.split("/"); - for (String part : logPathParts) { - if (part.startsWith("dt=")) { - logDate = new SimpleDateFormat("yyyy-MM-dd").parse(part.replaceFirst("dt=", "")); - break; - } + // 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) { - LOG.error("Did not find a date in the format yyyy-MM-dd in " + logFileName); + throw new Exception("Did not find a date in the format yyyy-MM-dd in " + logFileName); } + if (!s3AlterPathDate.after(logDate)) { return true; } @@ -111,7 +113,7 @@ public static boolean s3PathPrefixIsAltered(String logFileName, SecorConfig conf } public static String getS3AlternativePathPrefix(SecorConfig config) { - return config.getS3AlternativePath(); + return config.getS3AlternativePrefix(); } public static String getPrefix(String topic, SecorConfig config) throws IOException { From a3fa9ff8975f61dfc4dccaaee3a2651a5593de1a Mon Sep 17 00:00:00 2001 From: Josh Soref Date: Thu, 28 Apr 2016 05:29:26 +0000 Subject: [PATCH 169/330] Spelling fixes * spelling: commit * spelling: compatible * spelling: continuous * spelling: current * spelling: dimensional * spelling: excessive * spelling: generate * spelling: partition * spelling: properties * spelling: separate * spelling: storage * spelling: startup * spelling: system * spelling: transformer * spelling: separate (API) * spelling: receive (API) --- DESIGN.md | 2 +- README.md | 2 +- src/main/config/kafka.test.properties | 4 ++-- src/main/config/secor.common.properties | 6 +++--- src/main/config/secor.dev.azure.properties | 2 +- src/main/java/com/pinterest/secor/common/LogFilePath.java | 2 +- src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++-- .../com/pinterest/secor/parser/PartitionFinalizer.java | 2 +- src/main/java/com/pinterest/secor/parser/Partitioner.java | 2 +- .../pinterest/secor/parser/TimestampedMessageParser.java | 2 +- .../java/com/pinterest/secor/reader/MessageReader.java | 6 +++--- src/main/java/com/pinterest/secor/uploader/Uploader.java | 4 ++-- src/main/java/com/pinterest/secor/util/FileUtil.java | 4 ++-- .../java/com/pinterest/secor/util/ReflectionUtil.java | 8 ++++---- src/main/scripts/run_tests.sh | 2 +- src/test/java/com/pinterest/secor/util/FileUtilTest.java | 2 +- 16 files changed, 27 insertions(+), 27 deletions(-) diff --git a/DESIGN.md b/DESIGN.md index dd95bdbf9..6345e4cc0 100644 --- a/DESIGN.md +++ b/DESIGN.md @@ -160,7 +160,7 @@ 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. diff --git a/README.md b/README.md index 3bf89d110..a499f4c37 100644 --- a/README.md +++ b/README.md @@ -90,7 +90,7 @@ java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup. 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 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/secor.common.properties b/src/main/config/secor.common.properties index 0c79a419e..7b1a210b5 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -164,7 +164,7 @@ kafka.dual.commit.enabled=true # Possible values: "zookeeper" to read offset from zookeeper or "kafka" to read offset from kafka consumer topic kafka.offsets.storage=zookeeper -# Secor generation is a version that should be incremented during non-backwards-compabile +# 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 @@ -176,7 +176,7 @@ 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/... @@ -206,7 +206,7 @@ partitioner.finalizer.delay.seconds=3600 # The default is 10 # secor.finalizer.lookback.periods=10 -# If greater than 0, upon starup Secor will clean up directories and files under secor.local.path +# 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 diff --git a/src/main/config/secor.dev.azure.properties b/src/main/config/secor.dev.azure.properties index 871f4eeec..c9de7bc4e 100644 --- a/src/main/config/secor.dev.azure.properties +++ b/src/main/config/secor.dev.azure.properties @@ -15,7 +15,7 @@ secor.azure.endpoints.protocol=https secor.azure.account.name= secor.azure.account.key= -# Microsoft Azure blob sotrage container name. Container is a grouping of a set +# 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= diff --git a/src/main/java/com/pinterest/secor/common/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java index f73f8b6a9..71844d2de 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -27,7 +27,7 @@ * 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, diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index d03d92592..956ae181f 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -105,7 +105,7 @@ public String getFetchMessageMaxBytes() { return getString("kafka.fetch.message.max.bytes"); } - public String getSocketReceieveBufferBytes() { + public String getSocketReceiveBufferBytes() { return getString("kafka.socket.receive.buffer.bytes"); } @@ -151,7 +151,7 @@ public int getMessagesPerSecond() { public String getS3FileSystem() { return getString("secor.s3.filesystem"); } - public boolean getSeperateContainersForTopics() { + public boolean getSeparateContainersForTopics() { return getString("secor.swift.containers.for.each.topic").toLowerCase().equals("true"); } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 22bc19e0f..58dc60029 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -76,7 +76,7 @@ private String[] getFinalizedUptoPartitions(String topic) throws Exception { 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: {}, commmitted: {}", + LOG.error("For topic {} partition {}, lastMessage: {}, committed: {}", topicPartition.getTopic(), topicPartition.getPartition(), lastMessage, committedMessage); continue; diff --git a/src/main/java/com/pinterest/secor/parser/Partitioner.java b/src/main/java/com/pinterest/secor/parser/Partitioner.java index 6253758e0..1153e946f 100644 --- a/src/main/java/com/pinterest/secor/parser/Partitioner.java +++ b/src/main/java/com/pinterest/secor/parser/Partitioner.java @@ -40,7 +40,7 @@ * partitions to be finalized. * * The caller might repeat this loop multiple times when the filesystem partition - * is multi-dimentional (e.g. [dt=2015-07-07,hr=05]). it will loop once for the + * 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 diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index abedf628e..8a046f52f 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -196,7 +196,7 @@ public String[] getPreviousPartitions(String[] partitions) throws Exception { usingHourly = true; } } else if (mUsingHourly && millis % DAY_IN_MILLIS == 0) { - // On the day boundary, if the currrent partition is [dt=07-07, hr=00], the previous + // 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: diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index b9182ae2e..95f0c9db7 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -134,9 +134,9 @@ private ConsumerConfig createConsumerConfig() throws UnknownHostException { !mConfig.getRebalanceBackoffMs().isEmpty()) { props.put("rebalance.backoff.ms", mConfig.getRebalanceBackoffMs()); } - if (mConfig.getSocketReceieveBufferBytes() != null && - !mConfig.getSocketReceieveBufferBytes().isEmpty()) { - props.put("socket.receive.buffer.bytes", mConfig.getSocketReceieveBufferBytes()); + 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()); diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index cc984763d..a656c0d9f 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -80,9 +80,9 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { mZookeeperConnector.lock(lockPath); try { // Check if the committed offset has changed. - long zookeeperComittedOffsetCount = mZookeeperConnector.getCommittedOffsetCount( + long zookeeperCommittedOffsetCount = mZookeeperConnector.getCommittedOffsetCount( topicPartition); - if (zookeeperComittedOffsetCount == committedOffsetCount) { + 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); diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index ba75abc67..a28f485c9 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -118,7 +118,7 @@ public static String getPrefix(String topic, SecorConfig config) throws IOExcep String prefix = null; if (config.getCloudService().equals("Swift")) { String container = null; - if (config.getSeperateContainersForTopics()) { + if (config.getSeparateContainersForTopics()) { if (!exists("swift://" + topic + ".GENERICPROJECT")){ String containerUrl = "swift://" + topic + ".GENERICPROJECT"; Path containerPath = new Path(containerUrl); @@ -231,7 +231,7 @@ public static long getModificationTimeMsRecursive(String path) throws IOExceptio return modificationTime; } - /** Generat MD5 hash of topic and paritions. And extract first 4 characters of the MD5 hash. + /** Generate MD5 hash of topic and partitions. And extract first 4 characters of the MD5 hash. * @param topic * @param partitions * @return diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 2a14f7806..18d05daea 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -137,10 +137,10 @@ public static FileReader createFileReader(String className, LogFilePath logFileP } /** - * Create a MessageTrnasformer from it's fully qualified class name. The - * class passed in by name must be assignable to MessageTrnasformers and have - * 1-parameter constructor accepting a SecorConfig. Allows the MessageTrnasformers - * to be pluggable by providing the class name of a desired MessageTrnasformers in + * 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. diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 6d0217dfe..e41162e14 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -56,7 +56,7 @@ 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 filesytem +# 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 diff --git a/src/test/java/com/pinterest/secor/util/FileUtilTest.java b/src/test/java/com/pinterest/secor/util/FileUtilTest.java index e5f011677..185eb300b 100644 --- a/src/test/java/com/pinterest/secor/util/FileUtilTest.java +++ b/src/test/java/com/pinterest/secor/util/FileUtilTest.java @@ -34,7 +34,7 @@ public class FileUtilTest { public void setUp() throws Exception { mSwiftConfig = Mockito.mock(SecorConfig.class); Mockito.when(mSwiftConfig.getCloudService()).thenReturn("Swift"); - Mockito.when(mSwiftConfig.getSeperateContainersForTopics()).thenReturn(false); + Mockito.when(mSwiftConfig.getSeparateContainersForTopics()).thenReturn(false); Mockito.when(mSwiftConfig.getSwiftContainer()).thenReturn("some_container"); Mockito.when(mSwiftConfig.getSwiftPath()).thenReturn("some_swift_parent_dir"); From 21f492bb388d2e29987d5f27b698bec7a3c79942 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Mon, 2 May 2016 10:31:09 -0700 Subject: [PATCH 170/330] Cut version 0.17 and bump up the running snapshot version to 0.18-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 28a35f458..86c0c7214 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.17-SNAPSHOT + 0.18-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From ca0cf2343667eb9501557a68ef999d4b44418d4c Mon Sep 17 00:00:00 2001 From: Luca Milanesio Date: Thu, 28 Apr 2016 13:56:21 +0100 Subject: [PATCH 171/330] Make Uploader pluggable through secor.properties Allow custom logic to be applied to flush locally stored topic files to the underlying storage system. --- .../pinterest/secor/common/SecorConfig.java | 8 +++++ .../pinterest/secor/consumer/Consumer.java | 3 +- .../pinterest/secor/uploader/Uploader.java | 36 ++++++++++++++----- .../pinterest/secor/util/ReflectionUtil.java | 23 ++++++++++++ .../secor/uploader/UploaderTest.java | 2 +- 5 files changed, 62 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 956ae181f..dbbdb5842 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -215,6 +215,10 @@ 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"); } @@ -463,6 +467,10 @@ public String getString(String name) { return mProperties.getString(name); } + public String getString(String name, String defaultValue) { + return mProperties.getString(name, defaultValue); + } + public int getInt(String name) { checkProperty(name); return mProperties.getInt(name); diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 152cc4e6f..ee15fddd1 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -71,7 +71,8 @@ private void init() throws Exception { FileRegistry fileRegistry = new FileRegistry(mConfig); UploadManager uploadManager = ReflectionUtil.createUploadManager(mConfig.getUploadManagerClass(), mConfig); - mUploader = new Uploader(mConfig, mOffsetTracker, fileRegistry, uploadManager); + mUploader = ReflectionUtil.createUploader(mConfig.getUploaderClass()); + mUploader.init(mConfig, mOffsetTracker, fileRegistry, uploadManager); mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, fileRegistry); mMessageParser = ReflectionUtil.createMessageParser(mConfig.getMessageParserClass(), mConfig); mMessageTransformer = ReflectionUtil.createMessageTransformer(mConfig.getMessageTransformerClass(), mConfig); diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index a656c0d9f..daa1e090b 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -47,16 +47,25 @@ public class Uploader { private ZookeeperConnector mZookeeperConnector; private UploadManager mUploadManager; - public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, - UploadManager uploadManager) { - this(config, offsetTracker, fileRegistry, uploadManager, - 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 + */ + public void init(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, + UploadManager uploadManager) { + init(config, offsetTracker, fileRegistry, uploadManager, + new ZookeeperConnector(config)); } // For testing use only. - public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, - UploadManager uploadManager, - ZookeeperConnector zookeeperConnector) { + public void init(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, + UploadManager uploadManager, + ZookeeperConnector zookeeperConnector) { mConfig = config; mOffsetTracker = offsetTracker; mFileRegistry = fileRegistry; @@ -164,7 +173,7 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { } } - 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); @@ -201,6 +210,17 @@ 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) { diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 18d05daea..370748026 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -16,7 +16,9 @@ */ package com.pinterest.secor.util; +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.io.FileReader; import com.pinterest.secor.io.FileWriter; @@ -25,6 +27,7 @@ import com.pinterest.secor.transformer.MessageTransformer; import com.pinterest.secor.uploader.UploadManager; +import com.pinterest.secor.uploader.Uploader; import org.apache.hadoop.io.compress.CompressionCodec; /** @@ -60,6 +63,26 @@ public static UploadManager createUploadManager(String className, 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. diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index ca9a7a483..c83e069ce 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -52,7 +52,7 @@ public TestUploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, UploadManager uploadManager, ZookeeperConnector zookeeperConnector) { - super(config, offsetTracker, fileRegistry, uploadManager, zookeeperConnector); + init(config, offsetTracker, fileRegistry, uploadManager, zookeeperConnector); mReader = Mockito.mock(FileReader.class); } From 807fac4552442fac7e49376e37a1105c7088b743 Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Thu, 5 May 2016 23:22:56 -0700 Subject: [PATCH 172/330] Alter path return prefix only, rather than whole path. --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ---- src/main/java/com/pinterest/secor/util/FileUtil.java | 2 +- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index dbbdb5842..171d6ce2e 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -183,10 +183,6 @@ public String getS3Prefix() { return getS3FileSystem() + "://" + getS3Bucket() + "/" + getS3Path(); } - public String getS3AlternativePrefix() { - return getS3FileSystem() + "://" + getS3Bucket() + "/" + getS3AlternativePath(); - } - public String getLocalPath() { return getString("secor.local.path"); } diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index fa2cdbd97..f16ec5f9d 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -113,7 +113,7 @@ public static boolean s3PathPrefixIsAltered(String logFileName, SecorConfig conf } public static String getS3AlternativePathPrefix(SecorConfig config) { - return config.getS3AlternativePrefix(); + return config.getS3AlternativePath(); } public static String getPrefix(String topic, SecorConfig config) throws IOException { From e08abcaa3f90844c03e05ff9f2a8efa7a908ea7f Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Fri, 6 May 2016 10:44:21 -0700 Subject: [PATCH 173/330] Cut 0.18 and Bump up the version to 0.19-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 86c0c7214..4e7360703 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.18-SNAPSHOT + 0.19-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 90431fdb8cd5ec2d9c2277aac0db3fb608a606a2 Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Fri, 6 May 2016 14:58:48 -0700 Subject: [PATCH 174/330] update hadoop uploader to use full s3 path --- .../com/pinterest/secor/uploader/HadoopS3UploadManager.java | 2 +- src/main/java/com/pinterest/secor/util/FileUtil.java | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java index 387d64aa9..16771d1d2 100644 --- a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -50,7 +50,7 @@ public Handle upload(LogFilePath localPath) throws Exception { final String logFileName; if (FileUtil.s3PathPrefixIsAltered(path.getLogFilePath(), mConfig)) { - logFileName = localPath.withPrefix(FileUtil.getS3AlternativePathPrefix(mConfig)).getLogFilePath(); + logFileName = localPath.withPrefix(FileUtil.getS3AlternativePrefix(mConfig)).getLogFilePath(); LOG.info("Will upload file to alternative s3 prefix path {}", logFileName); } else { diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index f16ec5f9d..ad1fe6dac 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -116,6 +116,10 @@ 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")) { From 3f329c13dfc2e4dc41426d8f31951c56602aa6a7 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Sun, 8 May 2016 22:03:38 -0700 Subject: [PATCH 175/330] Cut 0.19 and bump up the version to 0.20-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4e7360703..96f405940 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.19-SNAPSHOT + 0.20-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 3c495ef1f1ca13f1200175191bbfe202f1b016d0 Mon Sep 17 00:00:00 2001 From: Luca Milanesio Date: Tue, 10 May 2016 15:29:50 +0100 Subject: [PATCH 176/330] Expose existing Log file writers Make the existing writers of the log files accessible from the outside as they may be needed for custom uploaders. --- .../com/pinterest/secor/common/FileRegistry.java | 10 ++++++++++ .../pinterest/secor/common/FileRegistryTest.java | 16 +++++++++++++++- 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index bd0249487..c5bafe776 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -75,6 +75,16 @@ public Collection getPaths(TopicPartition topicPartition) { 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. diff --git a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java index 021437cca..2f63b7089 100644 --- a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java +++ b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java @@ -64,7 +64,7 @@ public void setUp() throws Exception { mLogFilePathGz = new LogFilePath("/some_parent_dir", PATH_GZ); } - private void createWriter() throws Exception { + private FileWriter createWriter() throws Exception { PowerMockito.mockStatic(FileUtil.class); PowerMockito.mockStatic(ReflectionUtil.class); @@ -82,6 +82,8 @@ private void createWriter() throws Exception { FileWriter createdWriter = mRegistry.getOrCreateWriter( mLogFilePath, null); assertTrue(createdWriter == writer); + + return writer; } public void testGetOrCreateWriter() throws Exception { @@ -114,6 +116,18 @@ public void testGetOrCreateWriter() throws Exception { assertTrue(logFilePaths.contains(mLogFilePath)); } + 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); From ad3f7a209f5d9b6716441e6442d0ec2cb5f3a216 Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Tue, 10 May 2016 16:20:52 -0700 Subject: [PATCH 177/330] update finalizer to use full path prefix --- .../java/com/pinterest/secor/parser/PartitionFinalizer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 5ba122409..150b81f78 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -103,7 +103,7 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw mConfig.getGeneration(), 0, 0, mFileExtension); if (FileUtil.s3PathPrefixIsAltered(logFilePath.getLogFilePath(), mConfig)) { - logFilePath = logFilePath.withPrefix(FileUtil.getS3AlternativePathPrefix(mConfig)); + logFilePath = logFilePath.withPrefix(FileUtil.getS3AlternativePrefix(mConfig)); } String logFileDir = logFilePath.getLogFileDir(); @@ -182,7 +182,7 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw mConfig.getGeneration(), 0, 0, mFileExtension); if (FileUtil.s3PathPrefixIsAltered(logFilePath.getLogFilePath(), mConfig)) { - logFilePath = logFilePath.withPrefix(FileUtil.getS3AlternativePathPrefix(mConfig)); + logFilePath = logFilePath.withPrefix(FileUtil.getS3AlternativePrefix(mConfig)); LOG.info("Will finalize alternative s3 logFilePath {}", logFilePath); } From a3757810e0586e7e962593295ea95185f5c07400 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 11 May 2016 10:57:06 -0700 Subject: [PATCH 178/330] Cut version 0.20 and bump up running snapshot to 0.21-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 96f405940..52013f02d 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.20-SNAPSHOT + 0.21-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From d25222b461f1f3d62a60e446a9bc86c210362e0e Mon Sep 17 00:00:00 2001 From: Lovenish Goyal Date: Fri, 13 May 2016 23:34:54 +0530 Subject: [PATCH 179/330] update Readme.md --- README.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index a499f4c37..c42ab9303 100644 --- a/README.md +++ b/README.md @@ -9,10 +9,11 @@ Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud Storage - **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](e.g. 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 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 tranformation can be done by using customized class. - **Qubole interface**: Secor connects to [Qubole] to add finalized output partitions to Hive tables. ## Setup Guide @@ -122,6 +123,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Leo Woessner](https://github.com/estezz) * [Jerome Gagnon](https://github.com/jgagnon1) * [Taichi Nakashima](https://github.com/tcnksm) + * [Lovenish Goyal] (https://github.com/lovenishgoyal) ## Companies who use Secor From 37eb4620fa7387a7709916e4de343895906b3a67 Mon Sep 17 00:00:00 2001 From: Lovenish Goyal Date: Fri, 13 May 2016 23:41:12 +0530 Subject: [PATCH 180/330] update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index c42ab9303..7622167e4 100644 --- a/README.md +++ b/README.md @@ -9,7 +9,7 @@ Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud Storage - **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](e.g. day, hour, minute level partitions are supported by secor ) + - **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 performance properties are exposed through [Ostrich] and optionally exported to [OpenTSDB] / [statsD], - **customizability**: external log message parser may be loaded by updating the configuration, From d7e8e0467bff197452335563e8bbb84f6b5b59e8 Mon Sep 17 00:00:00 2001 From: Lovenish Goyal Date: Fri, 13 May 2016 23:42:07 +0530 Subject: [PATCH 181/330] update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 7622167e4..8cf56732d 100644 --- a/README.md +++ b/README.md @@ -9,7 +9,7 @@ Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud Storage - **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]. day,hour,minute level partitions are supported by secor) + - **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 performance properties are exposed through [Ostrich] and optionally exported to [OpenTSDB] / [statsD], - **customizability**: external log message parser may be loaded by updating the configuration, From c1a48fb956dd9c4d86c7688ebfdce86d8597c423 Mon Sep 17 00:00:00 2001 From: Dhruv Batheja Date: Wed, 18 May 2016 11:53:36 +0530 Subject: [PATCH 182/330] Removed absent parameters This inspection points out unresolved references inside javadoc --- src/main/java/com/pinterest/secor/consumer/Consumer.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index ee15fddd1..7a3d40856 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -167,9 +167,7 @@ private boolean consumeNextMessage() { /** * Helper to get the offset tracker (used in tests) * - * @param topic - * @param partition - * @return + * @return the offset tracker */ public OffsetTracker getOffsetTracker() { return this.mOffsetTracker; From 1367a9df4531c965e43cd4c9b7071bb60d75d46b Mon Sep 17 00:00:00 2001 From: zackloebelbegelman-ck Date: Wed, 8 Jun 2016 11:07:54 -0700 Subject: [PATCH 183/330] Add Credit Karma as a company who uses Secor We have been using Secor here at Credit Karma and wanted to add our name to the list of companies. --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 8cf56732d..12232f4e5 100644 --- a/README.md +++ b/README.md @@ -132,6 +132,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [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) From 9efe8309d35df167d47c97c60674d95247cf4950 Mon Sep 17 00:00:00 2001 From: Michael Spector Date: Thu, 23 Jun 2016 16:03:41 +0300 Subject: [PATCH 184/330] Added support for arbitrary (including nested) timestamp field in Google protobuf message --- pom.xml | 50 ++++++++++ .../pinterest/secor/common/SecorConfig.java | 2 + .../secor/parser/ProtobufMessageParser.java | 95 ++++++++++++++----- .../parser/ProtobufMessageParserTest.java | 79 +++++++++++---- src/test/protobuf/unittest.proto | 17 ++++ 5 files changed, 200 insertions(+), 43 deletions(-) create mode 100644 src/test/protobuf/unittest.proto diff --git a/pom.xml b/pom.xml index 52013f02d..1012e2a65 100644 --- a/pom.xml +++ b/pom.xml @@ -370,6 +370,56 @@ + + + com.github.os72 + protoc-jar-maven-plugin + 3.0.0-b3 + + + protobuf-test-sources + generate-test-sources + + run + + + 2.6.1 + + src/test/protobuf + + + src/test/protobuf + + + + 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 171d6ce2e..fe513b52c 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -437,6 +437,8 @@ public boolean getS3MD5HashPrefix() { public String getAzureContainer() { return getString("secor.azure.container.name"); } public String getAzurePath() { return getString("secor.azure.path"); } + + public String getProtobufMessageClass() { return getString("secor.protobuf.message.class"); } public TimeZone getTimeZone() { String timezone = getString("secor.parser.timezone"); diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index 03c3b0b93..c4606b59d 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -16,34 +16,85 @@ */ package com.pinterest.secor.parser; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.regex.Pattern; + +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.google.protobuf.CodedInputStream; - -import java.io.IOException; /** - * Basic protocol buffer parser. - * - * Assumes that the timestamp field is the first field, is required, - * and is a uint64. A more advanced parser might support an arbitrary - * field number (non-nested to keep things simple) and perhaps - * different data types. + * 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 { - public ProtobufMessageParser(SecorConfig config) { - super(config); - } - - @Override - public long extractTimestampMillis(final Message message) throws IOException { - CodedInputStream input = CodedInputStream.newInstance(message.getPayload()); - - // Don't really care about the tag, but need to read it to get - // to the payload. - int tag = input.readTag(); - return toMillis(input.readUInt64()); - } + + private static final Logger LOG = LoggerFactory.getLogger(ProtobufMessageParser.class); + + private Method messageParseMethod; + private String[] timestampFieldPath; + + public ProtobufMessageParser(SecorConfig config) { + super(config); + + String messageClassName = mConfig.getProtobufMessageClass(); + if (messageClassName != null) { + try { + Class messageClass = (Class) Class.forName(messageClassName); + messageParseMethod = messageClass.getDeclaredMethod("parseFrom", new Class[] { byte[].class }); + + String timestampFieldName = mConfig.getMessageTimestampName(); + String timestampFieldSeparator = mConfig.getMessageTimestampNameSeparator(); + if (timestampFieldSeparator == null || timestampFieldSeparator.isEmpty()) { + timestampFieldSeparator = "."; + } + timestampFieldPath = timestampFieldName.split(Pattern.quote(timestampFieldSeparator)); + } 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); + } + } + } + + @Override + public long extractTimestampMillis(final Message message) throws IOException { + if (messageParseMethod != null) { + com.google.protobuf.Message decodedMessage; + try { + decodedMessage = (com.google.protobuf.Message) messageParseMethod.invoke(null, message.getPayload()); + int i = 0; + for (; i < timestampFieldPath.length - 1; ++i) { + decodedMessage = (com.google.protobuf.Message) decodedMessage.getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); + } + return toMillis((Long) decodedMessage.getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i]))); + } catch (IllegalArgumentException e) { + throw new IOException("Unable to extract timestamp from protobuf message", e); + } catch (IllegalAccessException e) { + throw new IOException("Unable to extract timestamp from protobuf message", e); + } catch (InvocationTargetException e) { + throw new IOException("Unable to extract timestamp from protobuf message", e); + } + } else { + // Assume that the timestamp field is the first field, is required, + // and is a uint64. + + CodedInputStream input = CodedInputStream.newInstance(message.getPayload()); + // 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/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index af49a579f..8d23228be 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -18,6 +18,9 @@ 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; import org.junit.Test; import org.junit.runner.RunWith; @@ -27,25 +30,59 @@ @RunWith(PowerMockRunner.class) public class ProtobufMessageParserTest extends TestCase { - private SecorConfig mConfig; - - 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); - } - - @Override - public void setUp() throws Exception { - mConfig = Mockito.mock(SecorConfig.class); - } - - @Test - public void testExtractTimestampMillis() throws Exception { - ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); - - assertEquals(1405970352000l, parser.extractTimestampMillis(buildMessage(1405970352l))); - assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); - } + private SecorConfig mConfig; + + 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); + } + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + } + + @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 { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage1.class.getName()); + + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + UnitTestMessage1 message = UnitTestMessage1.newBuilder().setTimestamp(1405970352L).build(); + assertEquals(1405970352000l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + + message = UnitTestMessage1.newBuilder().setTimestamp(1405970352123l).build(); + assertEquals(1405970352123l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + } + + @Test + public void testExtractNestedTimestampMillis() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("internal.timestamp"); + Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage2.class.getName()); + + 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()))); + + message = UnitTestMessage2.newBuilder() + .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352123l).build()).build(); + assertEquals(1405970352123l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + } } diff --git a/src/test/protobuf/unittest.proto b/src/test/protobuf/unittest.proto new file mode 100644 index 000000000..7e58e3cdb --- /dev/null +++ b/src/test/protobuf/unittest.proto @@ -0,0 +1,17 @@ +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; +} From 3f1e8ce15acb1a306fef07bd3df17b36d4840bdd Mon Sep 17 00:00:00 2001 From: Michael Spector Date: Fri, 24 Jun 2016 13:05:26 +0300 Subject: [PATCH 185/330] Re-formatted using 4 spaces. Added logging. --- pom.xml | 22 +++++++++---------- .../secor/parser/ProtobufMessageParser.java | 18 ++++++++++----- .../parser/ProtobufMessageParserTest.java | 12 +++++----- 3 files changed, 29 insertions(+), 23 deletions(-) diff --git a/pom.xml b/pom.xml index 1012e2a65..4651b6c98 100644 --- a/pom.xml +++ b/pom.xml @@ -369,8 +369,8 @@ - - + + com.github.os72 protoc-jar-maven-plugin @@ -379,9 +379,9 @@ protobuf-test-sources generate-test-sources - - run - + + run + 2.6.1 @@ -391,12 +391,12 @@ src/test/protobuf - - java - none - target/generated-test-sources/protobuf/gen-java - - + + java + none + target/generated-test-sources/protobuf/gen-java + + diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index c4606b59d..93929d4f0 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -31,8 +31,10 @@ /** * 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. + * 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) */ @@ -45,7 +47,7 @@ public class ProtobufMessageParser extends TimestampedMessageParser { public ProtobufMessageParser(SecorConfig config) { super(config); - + String messageClassName = mConfig.getProtobufMessageClass(); if (messageClassName != null) { try { @@ -57,6 +59,8 @@ public ProtobufMessageParser(SecorConfig config) { if (timestampFieldSeparator == null || timestampFieldSeparator.isEmpty()) { timestampFieldSeparator = "."; } + LOG.info("Using protobuf timestamp field path: {} with separator: {}", timestampFieldName, + timestampFieldSeparator); timestampFieldPath = timestampFieldName.split(Pattern.quote(timestampFieldSeparator)); } catch (ClassNotFoundException e) { LOG.error("Unable to load protobuf message class", e); @@ -76,9 +80,11 @@ public long extractTimestampMillis(final Message message) throws IOException { decodedMessage = (com.google.protobuf.Message) messageParseMethod.invoke(null, message.getPayload()); int i = 0; for (; i < timestampFieldPath.length - 1; ++i) { - decodedMessage = (com.google.protobuf.Message) decodedMessage.getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); + decodedMessage = (com.google.protobuf.Message) decodedMessage + .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); } - return toMillis((Long) decodedMessage.getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i]))); + return toMillis((Long) decodedMessage + .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i]))); } catch (IllegalArgumentException e) { throw new IOException("Unable to extract timestamp from protobuf message", e); } catch (IllegalAccessException e) { @@ -89,7 +95,7 @@ public long extractTimestampMillis(final Message message) throws IOException { } else { // Assume that the timestamp field is the first field, is required, // and is a uint64. - + CodedInputStream input = CodedInputStream.newInstance(message.getPayload()); // Don't really care about the tag, but need to read it to get, to // the payload. diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index 8d23228be..2a9e077bf 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -61,11 +61,11 @@ public void testExtractPathTimestampMillis() throws Exception { UnitTestMessage1 message = UnitTestMessage1.newBuilder().setTimestamp(1405970352L).build(); assertEquals(1405970352000l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); message = UnitTestMessage1.newBuilder().setTimestamp(1405970352123l).build(); assertEquals(1405970352123l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); } @Test @@ -76,13 +76,13 @@ public void testExtractNestedTimestampMillis() throws Exception { ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); UnitTestMessage2 message = UnitTestMessage2.newBuilder() - .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352L).build()).build(); + .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352L).build()).build(); assertEquals(1405970352000l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); message = UnitTestMessage2.newBuilder() - .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352123l).build()).build(); + .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352123l).build()).build(); assertEquals(1405970352123l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); } } From d828e24cd0dc728285e406c1ed0ddcdb311f8c9f Mon Sep 17 00:00:00 2001 From: Michael Spector Date: Sat, 25 Jun 2016 10:07:11 +0300 Subject: [PATCH 186/330] Re-format using 4 spaces, now for real. --- .../secor/parser/ProtobufMessageParser.java | 114 +++++++++--------- .../parser/ProtobufMessageParserTest.java | 86 ++++++------- 2 files changed, 100 insertions(+), 100 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index 93929d4f0..a015fe41b 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -40,67 +40,67 @@ */ public class ProtobufMessageParser extends TimestampedMessageParser { - private static final Logger LOG = LoggerFactory.getLogger(ProtobufMessageParser.class); + private static final Logger LOG = LoggerFactory.getLogger(ProtobufMessageParser.class); - private Method messageParseMethod; - private String[] timestampFieldPath; + private Method messageParseMethod; + private String[] timestampFieldPath; - public ProtobufMessageParser(SecorConfig config) { - super(config); + public ProtobufMessageParser(SecorConfig config) { + super(config); - String messageClassName = mConfig.getProtobufMessageClass(); - if (messageClassName != null) { - try { - Class messageClass = (Class) Class.forName(messageClassName); - messageParseMethod = messageClass.getDeclaredMethod("parseFrom", new Class[] { byte[].class }); + String messageClassName = mConfig.getProtobufMessageClass(); + if (messageClassName != null) { + try { + Class messageClass = (Class) Class.forName(messageClassName); + messageParseMethod = messageClass.getDeclaredMethod("parseFrom", new Class[] { byte[].class }); - 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)); - } 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); - } - } - } + 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)); + } 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); + } + } + } - @Override - public long extractTimestampMillis(final Message message) throws IOException { - if (messageParseMethod != null) { - com.google.protobuf.Message decodedMessage; - try { - decodedMessage = (com.google.protobuf.Message) messageParseMethod.invoke(null, message.getPayload()); - int i = 0; - for (; i < timestampFieldPath.length - 1; ++i) { - decodedMessage = (com.google.protobuf.Message) decodedMessage - .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); - } - return toMillis((Long) decodedMessage - .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i]))); - } catch (IllegalArgumentException e) { - throw new IOException("Unable to extract timestamp from protobuf message", e); - } catch (IllegalAccessException e) { - throw new IOException("Unable to extract timestamp from protobuf message", e); - } catch (InvocationTargetException e) { - throw new IOException("Unable to extract timestamp from protobuf message", e); - } - } else { - // Assume that the timestamp field is the first field, is required, - // and is a uint64. + @Override + public long extractTimestampMillis(final Message message) throws IOException { + if (messageParseMethod != null) { + com.google.protobuf.Message decodedMessage; + try { + decodedMessage = (com.google.protobuf.Message) messageParseMethod.invoke(null, message.getPayload()); + int i = 0; + for (; i < timestampFieldPath.length - 1; ++i) { + decodedMessage = (com.google.protobuf.Message) decodedMessage + .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); + } + return toMillis((Long) decodedMessage + .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i]))); + } catch (IllegalArgumentException e) { + throw new IOException("Unable to extract timestamp from protobuf message", e); + } catch (IllegalAccessException e) { + throw new IOException("Unable to extract timestamp from protobuf message", e); + } catch (InvocationTargetException e) { + throw new IOException("Unable to extract timestamp from protobuf message", e); + } + } else { + // Assume that the timestamp field is the first field, is required, + // and is a uint64. - CodedInputStream input = CodedInputStream.newInstance(message.getPayload()); - // Don't really care about the tag, but need to read it to get, to - // the payload. - input.readTag(); - return toMillis(input.readUInt64()); - } - } + CodedInputStream input = CodedInputStream.newInstance(message.getPayload()); + // 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/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index 2a9e077bf..ba736a27d 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -30,59 +30,59 @@ @RunWith(PowerMockRunner.class) public class ProtobufMessageParserTest extends TestCase { - private SecorConfig mConfig; + private SecorConfig mConfig; - 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); - } + 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); + } - @Override - public void setUp() throws Exception { - mConfig = Mockito.mock(SecorConfig.class); - } + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + } - @Test - public void testExtractTimestampMillis() throws Exception { - ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + @Test + public void testExtractTimestampMillis() throws Exception { + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); - assertEquals(1405970352000l, parser.extractTimestampMillis(buildMessage(1405970352l))); - assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); - } + assertEquals(1405970352000l, parser.extractTimestampMillis(buildMessage(1405970352l))); + assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); + } - @Test - public void testExtractPathTimestampMillis() throws Exception { - Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); - Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage1.class.getName()); + @Test + public void testExtractPathTimestampMillis() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage1.class.getName()); - ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); - UnitTestMessage1 message = UnitTestMessage1.newBuilder().setTimestamp(1405970352L).build(); - assertEquals(1405970352000l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + UnitTestMessage1 message = UnitTestMessage1.newBuilder().setTimestamp(1405970352L).build(); + assertEquals(1405970352000l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); - message = UnitTestMessage1.newBuilder().setTimestamp(1405970352123l).build(); - assertEquals(1405970352123l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); - } + message = UnitTestMessage1.newBuilder().setTimestamp(1405970352123l).build(); + assertEquals(1405970352123l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + } - @Test - public void testExtractNestedTimestampMillis() throws Exception { - Mockito.when(mConfig.getMessageTimestampName()).thenReturn("internal.timestamp"); - Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage2.class.getName()); + @Test + public void testExtractNestedTimestampMillis() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("internal.timestamp"); + Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage2.class.getName()); - ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + 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()))); + UnitTestMessage2 message = UnitTestMessage2.newBuilder() + .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352L).build()).build(); + assertEquals(1405970352000l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); - message = UnitTestMessage2.newBuilder() - .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352123l).build()).build(); - assertEquals(1405970352123l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); - } + message = UnitTestMessage2.newBuilder() + .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352123l).build()).build(); + assertEquals(1405970352123l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + } } From 4918940e9ab62f1dd1ef2fe421dec7f272fdabd6 Mon Sep 17 00:00:00 2001 From: Yunjing Xu Date: Tue, 28 Jun 2016 12:40:31 -0700 Subject: [PATCH 187/330] Make secor.gs.credentials.path optional --- src/main/config/secor.common.properties | 5 +++++ .../java/com/pinterest/secor/uploader/GsUploadManager.java | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 7b1a210b5..a3723fe7b 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -113,6 +113,11 @@ secor.gs.path=data # 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 diff --git a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java index e5a762d58..26064444a 100644 --- a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java @@ -111,7 +111,7 @@ private static Storage getService(String credentialsPath, int connectTimeoutMs, GoogleCredential credential; try { // Lookup if configured path from the properties; otherwise fallback to Google Application default - if (credentialsPath != null) { + if (credentialsPath != null && !credentialsPath.isEmpty()) { credential = GoogleCredential .fromStream(new FileInputStream(credentialsPath), httpTransport, JSON_FACTORY) .createScoped(Collections.singleton(StorageScopes.CLOUD_PLATFORM)); From 0afef6bd673aea06b49a87dd8b1eda8541eeb0b2 Mon Sep 17 00:00:00 2001 From: Michael Spector Date: Wed, 29 Jun 2016 15:30:14 +0300 Subject: [PATCH 188/330] Added support for writing protobuf messages directly to Parquet --- pom.xml | 27 +++++ .../pinterest/secor/common/FileRegistry.java | 2 +- .../pinterest/secor/common/SecorConfig.java | 4 +- ...rotobufParquetFileReaderWriterFactory.java | 106 ++++++++++++++++++ .../secor/parser/ProtobufMessageParser.java | 59 ++++------ .../secor/tools/LogFileVerifier.java | 3 +- .../pinterest/secor/uploader/Uploader.java | 3 +- .../pinterest/secor/util/ProtobufUtil.java | 72 ++++++++++++ .../pinterest/secor/util/ReflectionUtil.java | 34 ++++-- .../secor/common/FileRegistryTest.java | 12 +- .../secor/io/FileReaderWriterFactoryTest.java | 18 +-- ...bufParquetFileReaderWriterFactoryTest.java | 78 +++++++++++++ .../parser/ProtobufMessageParserTest.java | 2 + .../secor/util/ReflectionUtilTest.java | 4 +- src/test/protobuf/unittest.proto | 6 + 15 files changed, 361 insertions(+), 69 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java create mode 100644 src/main/java/com/pinterest/secor/util/ProtobufUtil.java create mode 100644 src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java diff --git a/pom.xml b/pom.xml index 4651b6c98..32d46fd94 100644 --- a/pom.xml +++ b/pom.xml @@ -195,6 +195,33 @@ 2.7.0 + + + org.apache.parquet + parquet-common + 1.8.1 + + + org.apache.parquet + parquet-encoding + 1.8.1 + + + org.apache.parquet + parquet-column + 1.8.1 + + + org.apache.parquet + parquet-hadoop + 1.8.1 + + + org.apache.parquet + parquet-protobuf + 1.8.1 + + org.apache.thrift libthrift diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index c5bafe776..f78d4dede 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -109,7 +109,7 @@ public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) if (!files.contains(path)) { files.add(path); } - writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), path, codec); + 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()); diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index fe513b52c..441decfbb 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -439,7 +439,9 @@ public boolean getS3MD5HashPrefix() { public String getAzurePath() { return getString("secor.azure.path"); } public String getProtobufMessageClass() { return getString("secor.protobuf.message.class"); } - + + public boolean hasProtobufMessageClass() { return mProperties.containsKey("secor.protobuf.message.class"); } + public TimeZone getTimeZone() { String timezone = getString("secor.parser.timezone"); return Strings.isNullOrEmpty(timezone) ? TimeZone.getTimeZone("UTC") : TimeZone.getTimeZone(timezone); 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..ba55291e9 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java @@ -0,0 +1,106 @@ +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.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.proto.ProtoParquetReader; +import org.apache.parquet.proto.ProtoParquetWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.ProtobufUtil; + +/** + * Implementation for reading/writing protobuf messages to/from Parquet files. + * + * @author Michael Spector (spektom@gmail.com) + */ +public class ProtobufParquetFileReaderWriterFactory implements FileReaderWriterFactory { + + private static final Logger LOG = LoggerFactory.getLogger(ProtobufParquetFileReaderWriterFactory.class); + + private ProtobufUtil protobufUtil; + + public ProtobufParquetFileReaderWriterFactory(SecorConfig config) { + protobufUtil = new ProtobufUtil(config); + LOG.info("Using protobuf message class: {}" + config.getProtobufMessageClass()); + } + + @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; + + public ProtobufParquetFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + Path path = new Path(logFilePath.getLogFilePath()); + CompressionCodecName codecName = CompressionCodecName + .fromCompressionCodec(codec != null ? codec.getClass() : null); + writer = new ProtoParquetWriter(path, protobufUtil.getMessageClass(), codecName, + ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE); + } + + @Override + public long getLength() throws IOException { + return writer.getDataSize(); + } + + @Override + public void write(KeyValue keyValue) throws IOException { + Message message = protobufUtil.decodeMessage(keyValue.getValue()); + writer.write(message); + } + + @Override + public void close() throws IOException { + writer.close(); + } + } +} diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index a015fe41b..4d3d8b889 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -17,8 +17,6 @@ package com.pinterest.secor.parser; import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; import java.util.regex.Pattern; import org.slf4j.Logger; @@ -27,6 +25,7 @@ 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 @@ -42,56 +41,38 @@ public class ProtobufMessageParser extends TimestampedMessageParser { private static final Logger LOG = LoggerFactory.getLogger(ProtobufMessageParser.class); - private Method messageParseMethod; + private ProtobufUtil protobufUtil; private String[] timestampFieldPath; public ProtobufMessageParser(SecorConfig config) { super(config); - String messageClassName = mConfig.getProtobufMessageClass(); - if (messageClassName != null) { - try { - Class messageClass = (Class) Class.forName(messageClassName); - messageParseMethod = messageClass.getDeclaredMethod("parseFrom", new Class[] { byte[].class }); + if (config.hasProtobufMessageClass()) { + protobufUtil = new ProtobufUtil(config); + LOG.info("Using protobuf message class: {}" + config.getProtobufMessageClass()); - 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)); - } 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); + 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)); } } @Override public long extractTimestampMillis(final Message message) throws IOException { - if (messageParseMethod != null) { - com.google.protobuf.Message decodedMessage; - try { - decodedMessage = (com.google.protobuf.Message) messageParseMethod.invoke(null, message.getPayload()); - int i = 0; - for (; i < timestampFieldPath.length - 1; ++i) { - decodedMessage = (com.google.protobuf.Message) decodedMessage - .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); - } - return toMillis((Long) decodedMessage - .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i]))); - } catch (IllegalArgumentException e) { - throw new IOException("Unable to extract timestamp from protobuf message", e); - } catch (IllegalAccessException e) { - throw new IOException("Unable to extract timestamp from protobuf message", e); - } catch (InvocationTargetException e) { - throw new IOException("Unable to extract timestamp from protobuf message", e); + if (protobufUtil != null) { + com.google.protobuf.Message decodedMessage = protobufUtil.decodeMessage(message.getPayload()); + int i = 0; + for (; i < timestampFieldPath.length - 1; ++i) { + decodedMessage = (com.google.protobuf.Message) decodedMessage + .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); } + return toMillis((Long) decodedMessage + .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i]))); } else { // Assume that the timestamp field is the first field, is required, // and is a uint64. diff --git a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java index e7f585054..949d0c0c8 100644 --- a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java +++ b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java @@ -206,7 +206,8 @@ private FileReader createFileReader(LogFilePath logFilePath) throws Exception { FileReader fileReader = ReflectionUtil.createFileReader( mConfig.getFileReaderWriterFactory(), logFilePath, - codec + codec, + mConfig ); return fileReader; } diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index daa1e090b..af685a18a 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -120,7 +120,8 @@ protected FileReader createReader(LogFilePath srcPath, CompressionCodec codec) t return ReflectionUtil.createFileReader( mConfig.getFileReaderWriterFactory(), srcPath, - codec + codec, + mConfig ); } 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..90e4d1330 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/ProtobufUtil.java @@ -0,0 +1,72 @@ +package com.pinterest.secor.util; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + +import com.google.protobuf.Message; +import com.pinterest.secor.common.SecorConfig; + +/** + * Various utilities for working with protocol buffer encoded messages. + * + * @author Michael Spector (spektom@gmail.com) + */ +public class ProtobufUtil { + + private Class messageClass; + private Method messageParseMethod; + + /** + * 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) { + String messageClassName = config.getProtobufMessageClass(); + try { + messageClass = (Class) Class.forName(messageClassName); + messageParseMethod = messageClass.getDeclaredMethod("parseFrom", new Class[] { byte[].class }); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to load protobuf message class", e); + } catch (NoSuchMethodException e) { + throw new RuntimeException("Unable to find parseFrom() method in protobuf message class", e); + } catch (SecurityException e) { + throw new RuntimeException("Unable to use parseFrom() method from protobuf message class", e); + } + } + + /** + * @return protobuf message class used by this utility instance + */ + public Class getMessageClass() { + return messageClass; + } + + /** + * Decodes protobuf message + * + * @param payload + * Byte array containing encoded protobuf message + * @return protobuf message instance + * @throws RuntimeException + * when there's problem decoding protobuf message + */ + public Message decodeMessage(byte[] payload) { + try { + return (Message) messageParseMethod.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/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 370748026..76383463c 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -16,19 +16,17 @@ */ package com.pinterest.secor.util; -import com.pinterest.secor.common.FileRegistry; +import org.apache.hadoop.io.compress.CompressionCodec; + import com.pinterest.secor.common.LogFilePath; -import com.pinterest.secor.common.OffsetTracker; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.io.FileReader; -import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.io.FileWriter; 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 org.apache.hadoop.io.compress.CompressionCodec; /** * ReflectionUtil implements utility methods to construct objects of classes @@ -115,18 +113,26 @@ public static MessageParser createMessageParser(String className, * 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) throws Exception { + private static FileReaderWriterFactory createFileReaderWriterFactory(String className, + SecorConfig config) throws Exception { Class clazz = Class.forName(className); if (!FileReaderWriterFactory.class.isAssignableFrom(clazz)) { throw new IllegalArgumentException(String.format("The class '%s' is not assignable to '%s'.", className, FileReaderWriterFactory.class.getName())); } - // We assume a parameterless constructor - return (FileReaderWriterFactory) clazz.newInstance(); + 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(); + } } /** @@ -135,13 +141,15 @@ private static FileReaderWriterFactory createFileReaderWriterFactory(String clas * @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) + CompressionCodec codec, + SecorConfig config) throws Exception { - return createFileReaderWriterFactory(className).BuildFileWriter(logFilePath, codec); + return createFileReaderWriterFactory(className, config).BuildFileWriter(logFilePath, codec); } /** @@ -150,13 +158,15 @@ public static FileWriter createFileWriter(String className, LogFilePath logFileP * @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) + CompressionCodec codec, + SecorConfig config) throws Exception { - return createFileReaderWriterFactory(className).BuildFileReader(logFilePath, codec); + return createFileReaderWriterFactory(className, config).BuildFileReader(logFilePath, codec); } /** diff --git a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java index 2f63b7089..d7a08ce6f 100644 --- a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java +++ b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java @@ -73,7 +73,8 @@ private FileWriter createWriter() throws Exception { ReflectionUtil.createFileWriter( Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class) + Mockito.any(CompressionCodec.class), + Mockito.any(SecorConfig.class) )) .thenReturn(writer); @@ -96,7 +97,8 @@ public void testGetOrCreateWriter() throws Exception { PowerMockito.verifyStatic(); ReflectionUtil.createFileWriter(Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class) + Mockito.any(CompressionCodec.class), + Mockito.any(SecorConfig.class) ); PowerMockito.verifyStatic(); @@ -137,7 +139,8 @@ private void createCompressedWriter() throws Exception { ReflectionUtil.createFileWriter( Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class) + Mockito.any(CompressionCodec.class), + Mockito.any(SecorConfig.class) )) .thenReturn(writer); @@ -162,7 +165,8 @@ public void testGetOrCreateWriterCompressed() throws Exception { PowerMockito.verifyStatic(); ReflectionUtil.createFileWriter(Mockito.any(String.class), Mockito.any(LogFilePath.class), - Mockito.any(CompressionCodec.class) + Mockito.any(CompressionCodec.class), + Mockito.any(SecorConfig.class) ); TopicPartition topicPartition = new TopicPartition("some_topic", 0); diff --git a/src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java index 0ef5b5742..54896d63a 100644 --- a/src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java +++ b/src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java @@ -173,14 +173,15 @@ private void mockSequenceFileWriter(boolean isCompressed) public void testSequenceFileReader() throws Exception { setupSequenceFileReaderConfig(); mockSequenceFileWriter(false); - ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePath, null); + 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()); + ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), mLogFilePathGz, new GzipCodec(), + mConfig); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); @@ -193,7 +194,7 @@ public void testSequenceFileWriter() throws Exception { mockSequenceFileWriter(false); FileWriter writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), - mLogFilePath, null); + mLogFilePath, null, mConfig); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); @@ -205,7 +206,7 @@ public void testSequenceFileWriter() throws Exception { mockSequenceFileWriter(true); writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), - mLogFilePathGz, new GzipCodec()); + mLogFilePathGz, new GzipCodec(), mConfig); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); @@ -221,14 +222,14 @@ public void testDelimitedTextFileWriter() throws Exception { mockDelimitedTextFileWriter(false); FileWriter writer = (FileWriter) ReflectionUtil .createFileWriter(mConfig.getFileReaderWriterFactory(), - mLogFilePath, null + mLogFilePath, null, mConfig ); assert writer.getLength() == 0L; mockDelimitedTextFileWriter(true); writer = (FileWriter) ReflectionUtil .createFileWriter(mConfig.getFileReaderWriterFactory(), - mLogFilePathGz, new GzipCodec() + mLogFilePathGz, new GzipCodec(), mConfig ); assert writer.getLength() == 0L; } @@ -238,9 +239,10 @@ public void testDelimitedTextFileReader() throws Exception { mockDelimitedTextFileWriter(false); - ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePath, null); + ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePath, null, mConfig); mockDelimitedTextFileWriter(true); - ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePathGz, new GzipCodec()); + 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/ProtobufParquetFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java new file mode 100644 index 000000000..39553d8e8 --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.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 static org.junit.Assert.assertArrayEquals; + +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.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 { + Mockito.when(config.getProtobufMessageClass()).thenReturn(UnitTestMessage3.class.getName()); + Mockito.when(config.getFileReaderWriterFactory()).thenReturn(ProtobufParquetFileReaderWriterFactory.class.getName()); + + 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/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index ba736a27d..b84a4887c 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -55,6 +55,7 @@ public void testExtractTimestampMillis() throws Exception { @Test public void testExtractPathTimestampMillis() throws Exception { Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.hasProtobufMessageClass()).thenReturn(true); Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage1.class.getName()); ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); @@ -71,6 +72,7 @@ public void testExtractPathTimestampMillis() throws Exception { @Test public void testExtractNestedTimestampMillis() throws Exception { Mockito.when(mConfig.getMessageTimestampName()).thenReturn("internal.timestamp"); + Mockito.when(mConfig.hasProtobufMessageClass()).thenReturn(true); Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage2.class.getName()); ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java index 4b74650c8..34ba65fa6 100644 --- a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java +++ b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java @@ -50,7 +50,7 @@ public void testMessageParserClassNotFound() throws Exception { @Test(expected = ClassNotFoundException.class) public void testFileWriterClassNotFound() throws Exception { - ReflectionUtil.createFileWriter("com.example.foo", mLogFilePath, null); + ReflectionUtil.createFileWriter("com.example.foo", mLogFilePath, null, mSecorConfig); } @Test(expected = IllegalArgumentException.class) @@ -66,6 +66,6 @@ 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); + mLogFilePath, null, mSecorConfig); } } \ No newline at end of file diff --git a/src/test/protobuf/unittest.proto b/src/test/protobuf/unittest.proto index 7e58e3cdb..e21182da9 100644 --- a/src/test/protobuf/unittest.proto +++ b/src/test/protobuf/unittest.proto @@ -15,3 +15,9 @@ message UnitTestMessage2 { optional Internal internal = 1; } + +message UnitTestMessage3 { + optional string data = 1; + optional uint64 timestamp = 2; + optional bytes _secor_metadata = 3; +} \ No newline at end of file From cf148bed2a93eb67a85cd3c8f16849b79e30e0f9 Mon Sep 17 00:00:00 2001 From: Michael Spector Date: Wed, 29 Jun 2016 21:38:40 +0300 Subject: [PATCH 189/330] Added documentation about Protobuf date parser. Added Appsflyer to the list of companies. --- README.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 12232f4e5..579daa753 100644 --- a/README.md +++ b/README.md @@ -52,11 +52,13 @@ 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 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. +- **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 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. -- **msgpack 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. +- **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. 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. @@ -138,6 +140,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Nextperf](http://www.nextperf.com) * [Zalando](http://www.zalando.com) * [Rakuten](http://techblog.rakuten.co.jp/) + * [Appsflyer](https://www.appsflyer.com) ## Help From 620eb566b95719658a8ac45e2802eecd2fbcd834 Mon Sep 17 00:00:00 2001 From: Michael Spector Date: Wed, 29 Jun 2016 21:39:44 +0300 Subject: [PATCH 190/330] Added documentation about Protobuf date parser. Added Appsflyer to the list of companies --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 579daa753..2327aaf7f 100644 --- a/README.md +++ b/README.md @@ -58,7 +58,7 @@ One of the convenience features of Secor is the ability to group messages and sa - **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. +- **[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. 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. @@ -157,3 +157,4 @@ If you have any questions or comments, you can reach us at [secor-users@googlegr [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/ From 7071f1b89587760c7e18685e08d7dfc26f9edfb7 Mon Sep 17 00:00:00 2001 From: Michael Spector Date: Wed, 29 Jun 2016 22:49:48 +0300 Subject: [PATCH 191/330] Removed unneeded field. --- src/test/protobuf/unittest.proto | 1 - 1 file changed, 1 deletion(-) diff --git a/src/test/protobuf/unittest.proto b/src/test/protobuf/unittest.proto index e21182da9..6d975fadd 100644 --- a/src/test/protobuf/unittest.proto +++ b/src/test/protobuf/unittest.proto @@ -19,5 +19,4 @@ message UnitTestMessage2 { message UnitTestMessage3 { optional string data = 1; optional uint64 timestamp = 2; - optional bytes _secor_metadata = 3; } \ No newline at end of file From 89c0c0b8163db550db4c5fb1aa3b45c9ce7e4282 Mon Sep 17 00:00:00 2001 From: Michael Spector Date: Sun, 3 Jul 2016 09:22:26 +0300 Subject: [PATCH 192/330] Specify protobuf message class per Kafka topic or globally using '*' --- .../pinterest/secor/common/SecorConfig.java | 16 +++- ...rotobufParquetFileReaderWriterFactory.java | 11 +-- .../secor/parser/ProtobufMessageParser.java | 14 +-- .../pinterest/secor/util/ProtobufUtil.java | 88 +++++++++++++++---- .../config/secor.test.protobuf.properties | 2 + .../secor/common/SecorConfigTest.java | 18 ++++ ...bufParquetFileReaderWriterFactoryTest.java | 24 +++-- .../parser/ProtobufMessageParserTest.java | 24 +++-- 8 files changed, 146 insertions(+), 51 deletions(-) create mode 100644 src/test/config/secor.test.protobuf.properties diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 441decfbb..d50623973 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -21,6 +21,8 @@ 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; @@ -438,9 +440,17 @@ public boolean getS3MD5HashPrefix() { public String getAzurePath() { return getString("secor.azure.path"); } - public String getProtobufMessageClass() { return getString("secor.protobuf.message.class"); } - - public boolean hasProtobufMessageClass() { return mProperties.containsKey("secor.protobuf.message.class"); } + 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 TimeZone getTimeZone() { String timezone = getString("secor.parser.timezone"); diff --git a/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java index ba55291e9..9129a68b1 100644 --- a/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java @@ -9,8 +9,6 @@ import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.proto.ProtoParquetReader; import org.apache.parquet.proto.ProtoParquetWriter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import com.google.protobuf.Message; import com.google.protobuf.Message.Builder; @@ -30,13 +28,10 @@ */ public class ProtobufParquetFileReaderWriterFactory implements FileReaderWriterFactory { - private static final Logger LOG = LoggerFactory.getLogger(ProtobufParquetFileReaderWriterFactory.class); - private ProtobufUtil protobufUtil; public ProtobufParquetFileReaderWriterFactory(SecorConfig config) { protobufUtil = new ProtobufUtil(config); - LOG.info("Using protobuf message class: {}" + config.getProtobufMessageClass()); } @Override @@ -78,12 +73,14 @@ public void close() throws IOException { 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); - writer = new ProtoParquetWriter(path, protobufUtil.getMessageClass(), codecName, + topic = logFilePath.getTopic(); + writer = new ProtoParquetWriter(path, protobufUtil.getMessageClass(topic), codecName, ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE); } @@ -94,7 +91,7 @@ public long getLength() throws IOException { @Override public void write(KeyValue keyValue) throws IOException { - Message message = protobufUtil.decodeMessage(keyValue.getValue()); + Message message = protobufUtil.decodeMessage(topic, keyValue.getValue()); writer.write(message); } diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index 4d3d8b889..afc2af7bf 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -47,10 +47,8 @@ public class ProtobufMessageParser extends TimestampedMessageParser { public ProtobufMessageParser(SecorConfig config) { super(config); - if (config.hasProtobufMessageClass()) { - protobufUtil = new ProtobufUtil(config); - LOG.info("Using protobuf message class: {}" + config.getProtobufMessageClass()); - + protobufUtil = new ProtobufUtil(config); + if (protobufUtil.isConfigured()) { String timestampFieldName = mConfig.getMessageTimestampName(); String timestampFieldSeparator = mConfig.getMessageTimestampNameSeparator(); if (timestampFieldSeparator == null || timestampFieldSeparator.isEmpty()) { @@ -59,13 +57,17 @@ public ProtobufMessageParser(SecorConfig config) { 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 { - if (protobufUtil != null) { - com.google.protobuf.Message decodedMessage = protobufUtil.decodeMessage(message.getPayload()); + if (timestampFieldPath != null) { + com.google.protobuf.Message decodedMessage = protobufUtil.decodeMessage(message.getTopic(), + message.getPayload()); int i = 0; for (; i < timestampFieldPath.length - 1; ++i) { decodedMessage = (com.google.protobuf.Message) decodedMessage diff --git a/src/main/java/com/pinterest/secor/util/ProtobufUtil.java b/src/main/java/com/pinterest/secor/util/ProtobufUtil.java index 90e4d1330..b7572cb88 100644 --- a/src/main/java/com/pinterest/secor/util/ProtobufUtil.java +++ b/src/main/java/com/pinterest/secor/util/ProtobufUtil.java @@ -2,19 +2,39 @@ 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. + * 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 Class messageClass; - private Method messageParseMethod; + 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} @@ -27,38 +47,70 @@ public class ProtobufUtil { */ @SuppressWarnings("unchecked") public ProtobufUtil(SecorConfig config) { - String messageClassName = config.getProtobufMessageClass(); - try { - messageClass = (Class) Class.forName(messageClassName); - messageParseMethod = messageClass.getDeclaredMethod("parseFrom", new Class[] { byte[].class }); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Unable to load protobuf message class", e); - } catch (NoSuchMethodException e) { - throw new RuntimeException("Unable to find parseFrom() method in protobuf message class", e); - } catch (SecurityException e) { - throw new RuntimeException("Unable to use parseFrom() method from protobuf message class", e); + 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); + } } } /** - * @return protobuf message class used by this utility instance + * 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() { - return messageClass; + 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(byte[] payload) { + public Message decodeMessage(String topic, byte[] payload) { try { - return (Message) messageParseMethod.invoke(null, payload); + 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); 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/java/com/pinterest/secor/common/SecorConfigTest.java b/src/test/java/com/pinterest/secor/common/SecorConfigTest.java index 513763238..b8d507aa3 100644 --- a/src/test/java/com/pinterest/secor/common/SecorConfigTest.java +++ b/src/test/java/com/pinterest/secor/common/SecorConfigTest.java @@ -3,7 +3,13 @@ import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.configuration.PropertiesConfiguration; import org.junit.Test; + +import com.pinterest.secor.protobuf.Messages.UnitTestMessage1; +import com.pinterest.secor.protobuf.Messages.UnitTestMessage2; + import java.net.URL; +import java.util.Map; + import static org.junit.Assert.*; public class SecorConfigTest { @@ -30,5 +36,17 @@ public void config_should_read_migration_required() throws ConfigurationExceptio 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")); + } } diff --git a/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java index 39553d8e8..19f2b4892 100644 --- a/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java +++ b/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java @@ -18,6 +18,9 @@ import static org.junit.Assert.assertArrayEquals; +import java.util.HashMap; +import java.util.Map; + import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mockito; @@ -46,13 +49,17 @@ public void setUp() throws Exception { @Test public void testProtobufParquetReadWriteRoundTrip() throws Exception { - Mockito.when(config.getProtobufMessageClass()).thenReturn(UnitTestMessage3.class.getName()); - Mockito.when(config.getFileReaderWriterFactory()).thenReturn(ProtobufParquetFileReaderWriterFactory.class.getName()); - + 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()); + 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); + + 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(); @@ -63,13 +70,14 @@ public void testProtobufParquetReadWriteRoundTrip() throws Exception { fileWriter.write(kv2); fileWriter.close(); - FileReader fileReader = ReflectionUtil.createFileReader(config.getFileReaderWriterFactory(), tempLogFilePath, null, config); - + 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()); diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index b84a4887c..043bcdf70 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -16,17 +16,21 @@ */ package com.pinterest.secor.parser; -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 java.util.HashMap; +import java.util.Map; -import junit.framework.TestCase; 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 { @@ -54,9 +58,10 @@ public void testExtractTimestampMillis() throws Exception { @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.hasProtobufMessageClass()).thenReturn(true); - Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage1.class.getName()); + Mockito.when(mConfig.getProtobufMessageClassPerTopic()).thenReturn(classPerTopic); ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); @@ -71,9 +76,10 @@ public void testExtractPathTimestampMillis() throws Exception { @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.hasProtobufMessageClass()).thenReturn(true); - Mockito.when(mConfig.getProtobufMessageClass()).thenReturn(UnitTestMessage2.class.getName()); + Mockito.when(mConfig.getProtobufMessageClassPerTopic()).thenReturn(classPerTopic); ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); From 00751668d5b8ed512ef35de2d1f8c991fef6095d Mon Sep 17 00:00:00 2001 From: Michael Spector Date: Mon, 4 Jul 2016 15:49:45 +0300 Subject: [PATCH 193/330] Added documentation on Parquet output format --- README.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 2327aaf7f..f3e394503 100644 --- a/README.md +++ b/README.md @@ -68,9 +68,11 @@ If none of the parsers available out-of-the-box is suitable for your use case, n Currently secor supports the following output formats -- **Sequence Files**: Flat file containing binary key value pairs. +- **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. +- **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. + +- **[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.*=`. ## Tools Secor comes with a number of tools implementing interactions with the environment. @@ -158,3 +160,4 @@ If you have any questions or comments, you can reach us at [secor-users@googlegr [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/ From 01c25633ebcc68e3e6ad1cb7464eafbf0071e64a Mon Sep 17 00:00:00 2001 From: Prasanna Gautam Date: Tue, 19 Jul 2016 11:47:36 -0700 Subject: [PATCH 194/330] Adding argument reading to LogVerifier It was failing with null because it wasn't looking for an arg. --- src/main/java/com/pinterest/secor/main/LogFileVerifierMain.java | 2 ++ 1 file changed, 2 insertions(+) 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")); From f461ac0e55e8e1a79bd4b06d43bd34ccccceb8a5 Mon Sep 17 00:00:00 2001 From: Prasanna Gautam Date: Thu, 28 Jul 2016 17:20:53 -0700 Subject: [PATCH 195/330] Uploads at minute mark for selected topics - This is restricted to particular topic filters that may require files to be available at particular time for batch jobs - Adds two configs: - secor.kafka.upload_at_minute_mark.topic_filter - secor.upload.minute_mark --- src/main/config/secor.common.properties | 6 +++ .../pinterest/secor/common/SecorConfig.java | 4 ++ .../pinterest/secor/uploader/Uploader.java | 22 +++++++++- .../secor/uploader/UploaderTest.java | 44 ++++++++++++++++++- 4 files changed, 74 insertions(+), 2 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index a3723fe7b..30bea207e 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -307,3 +307,9 @@ secor.s3.alter.path.date= # An alternative S3 path for secor to upload files to secor.s3.alternative.path= + +# 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index d50623973..edaab6f95 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -197,6 +197,10 @@ 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"); } diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index af685a18a..ca8f5c6ba 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -27,6 +27,7 @@ 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; @@ -181,12 +182,31 @@ protected void trimFiles(TopicPartition topicPartition, long startOffset) throws } } + /*** + * 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(); + final String topicFilter = mConfig.getKafkaTopicUploadAtMinuteMarkFilter(); + if (topicFilter == null || topicFilter.isEmpty()){ return false; } + if (topic.matches(topicFilter)){ + if (DateTime.now().minuteOfHour().get() == mConfig.getUploadMinuteMark()){ + return true; + } + } + return false; + } + private 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); diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index c83e069ce..74e929660 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -26,8 +26,11 @@ import junit.framework.TestCase; 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; @@ -43,7 +46,7 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ @RunWith(PowerMockRunner.class) -@PrepareForTest({ FileUtil.class, IdUtil.class }) +@PrepareForTest({ FileUtil.class, IdUtil.class , DateTime.class}) public class UploaderTest extends TestCase { private static class TestUploader extends Uploader { private FileReader mReader; @@ -109,6 +112,44 @@ public void setUp() throws Exception { 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)) @@ -125,6 +166,7 @@ public void testUploadFiles() throws Exception { 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"); From ad0044b6bf10cf8415e6b8ef2ff761e1181dbce9 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 2 Aug 2016 15:16:55 -0700 Subject: [PATCH 196/330] File Coalescing: Enhance FileRegistry and LogFilePath to have the capability of specifying multiple partitions We want to achieve a file coalescing feature currently in Spark streaming, for slow volume topic or if you upload by minute, we want to reduce the number of files we are generating on S3/HDFS side. Enhance the FileRegistry and LogFilePath to allow a file path represent multiple kafka partitions encoded as: topic/dt=XX/hr=XX/generationNumber_startKafkaPartition-endKafkaPartition_MD5-encoded-partition-offsets --- .../pinterest/secor/common/FileRegistry.java | 83 ++++++++---- .../pinterest/secor/common/LogFilePath.java | 120 +++++++++++++----- .../secor/common/TopicPartitionGroup.java | 84 ++++++++++++ .../impl/SequenceFileReaderWriterFactory.java | 10 +- 4 files changed, 240 insertions(+), 57 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/common/TopicPartitionGroup.java diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index f78d4dede..fe43a5151 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -38,13 +38,13 @@ public class FileRegistry { private static final Logger LOG = LoggerFactory.getLogger(FileRegistry.class); private final SecorConfig mConfig; - private HashMap> mFiles; + private HashMap> mFiles; private HashMap mWriters; private HashMap mCreationTimes; public FileRegistry(SecorConfig mConfig) { this.mConfig = mConfig; - mFiles = new HashMap>(); + mFiles = new HashMap>(); mWriters = new HashMap(); mCreationTimes = new HashMap(); } @@ -54,12 +54,23 @@ public FileRegistry(SecorConfig mConfig) { * @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 a copy of the collection to prevent the caller from modifying internals. - return new HashSet(topicPartitions); + return tps; + } + + public Collection getTopicPartitionGroups() { + Set topicPartitions = mFiles.keySet(); + Set tps = new HashSet(); + if (topicPartitions != null) { + tps.addAll(topicPartitions); + } + return tps; } /** @@ -68,7 +79,16 @@ 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(); } @@ -99,8 +119,8 @@ public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) // 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(); @@ -128,16 +148,16 @@ public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) * @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()); @@ -150,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; } @@ -181,9 +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 {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); + LOG.warn("No paths found for topic {} partition {}", topicPartitionGroup.getTopic(), + Arrays.toString(topicPartitionGroup.getPartitions())); } else { for (LogFilePath path : paths) { deleteWriter(path); @@ -199,7 +228,11 @@ 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) { FileWriter writer = mWriters.get(path); @@ -207,8 +240,8 @@ public long getSize(TopicPartition topicPartition) throws IOException { 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; } @@ -220,9 +253,13 @@ 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); + Collection paths = getPaths(topicPartitionGroup); for (LogFilePath path : paths) { Long creationTime = mCreationTimes.get(path); if (creationTime == null) { @@ -237,8 +274,8 @@ public long getModificationAgeSec(TopicPartition topicPartition) throws IOExcept 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/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java index 71844d2de..3d9aa8f9c 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -17,8 +17,12 @@ 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; @@ -46,38 +50,51 @@ public class LogFilePath { private final String mTopic; private final String[] mPartitions; private final int mGeneration; - private final int mKafkaPartition; - private final long mOffset; + private final int[] mKafkaPartitions; + private final long[] mOffsets; private final String mExtension; + 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,12 +127,21 @@ 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, mKafkaPartition, mOffset, mExtension); + return new LogFilePath(prefix, mTopic, mPartitions, mGeneration, mKafkaPartitions, mOffsets, + mExtension); } public String getLogFileParentDir() { @@ -137,8 +163,26 @@ 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, "_"); } @@ -174,12 +218,22 @@ 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() { @@ -194,8 +248,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; @@ -209,8 +263,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/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/io/impl/SequenceFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java index 0bb56a77b..88492d5c2 100644 --- a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java @@ -29,6 +29,8 @@ 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; @@ -40,6 +42,9 @@ * @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); @@ -83,10 +88,11 @@ 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(); - Path fsPath = new Path(path.getLogFilePath()); + fsPath = new Path(path.getLogFilePath()); FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); if (codec != null) { this.mWriter = SequenceFile.createWriter(fs, config, fsPath, @@ -98,6 +104,7 @@ public SequenceFileWriter(LogFilePath path, CompressionCodec codec) throws IOExc } this.mKey = new LongWritable(); this.mValue = new BytesWritable(); + LOG.info("Created sequence file writer: {}", fsPath); } @Override @@ -115,6 +122,7 @@ public void write(KeyValue keyValue) throws IOException { @Override public void close() throws IOException { this.mWriter.close(); + LOG.info("Closing sequence file writer: {}", fsPath); } } } \ No newline at end of file From 0008f77a4c757d3438d701171724d50e9e4331f5 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 2 Aug 2016 23:17:48 -0700 Subject: [PATCH 197/330] Bump up the version to 0.21 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 32d46fd94..351c35fd4 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.21-SNAPSHOT + 0.22-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 558a6ab61030369f98476c1e8f3c9ceee4411c2a Mon Sep 17 00:00:00 2001 From: Artem Golotin Date: Wed, 3 Aug 2016 14:45:05 -0600 Subject: [PATCH 198/330] Get the Iso8601 message parser to create correct log instance #243 --- .../java/com/pinterest/secor/parser/Iso8601MessageParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java index 2e187b53c..7b591eaf4 100644 --- a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java @@ -36,7 +36,7 @@ * */ public class Iso8601MessageParser extends MessageParser { - private static final Logger LOG = LoggerFactory.getLogger(DateMessageParser.class); + private static final Logger LOG = LoggerFactory.getLogger(Iso8601MessageParser.class); protected static final String defaultDate = "dt=1970-01-01"; protected static final String defaultFormatter = "yyyy-MM-dd"; protected static final SimpleDateFormat outputFormatter = new SimpleDateFormat(defaultFormatter); From f7a0c7b2be54a281433d46b6eea510e7bf48a06b Mon Sep 17 00:00:00 2001 From: Alex Glikson Date: Thu, 4 Aug 2016 18:51:18 +0300 Subject: [PATCH 199/330] Minor DESIGN.md wording enhancement --- DESIGN.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/DESIGN.md b/DESIGN.md index 6345e4cc0..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 rely 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 From 42f1e221661128c1267bea838cc1ff931451e219 Mon Sep 17 00:00:00 2001 From: Prasanna Gautam Date: Mon, 8 Aug 2016 13:41:14 -0700 Subject: [PATCH 200/330] doing topicFilter parsing at Class init --- .../java/com/pinterest/secor/uploader/Uploader.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index ca8f5c6ba..37dc7add7 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -47,6 +47,7 @@ public class Uploader { private FileRegistry mFileRegistry; private ZookeeperConnector mZookeeperConnector; private UploadManager mUploadManager; + private String mTopicFilter; /** @@ -72,6 +73,8 @@ public void init(SecorConfig config, OffsetTracker offsetTracker, FileRegistry f mFileRegistry = fileRegistry; mUploadManager = uploadManager; mZookeeperConnector = zookeeperConnector; + mTopicFilter = mConfig.getKafkaTopicUploadAtMinuteMarkFilter(); + } private void uploadFiles(TopicPartition topicPartition) throws Exception { @@ -190,9 +193,10 @@ protected void trimFiles(TopicPartition topicPartition, long startOffset) throws */ private boolean isRequiredToUploadAtTime(TopicPartition topicPartition) throws Exception{ final String topic = topicPartition.getTopic(); - final String topicFilter = mConfig.getKafkaTopicUploadAtMinuteMarkFilter(); - if (topicFilter == null || topicFilter.isEmpty()){ return false; } - if (topic.matches(topicFilter)){ + if (mTopicFilter == null || mTopicFilter.isEmpty()){ + return false; + } + if (topic.matches(mTopicFilter)){ if (DateTime.now().minuteOfHour().get() == mConfig.getUploadMinuteMark()){ return true; } From bd62a7101726ce61d725c00c12b0d9a60666f86b Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Mon, 3 Oct 2016 15:37:27 -0700 Subject: [PATCH 201/330] added timeout to qubole calls --- .../com/pinterest/secor/parser/QuboleClient.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/QuboleClient.java b/src/main/java/com/pinterest/secor/parser/QuboleClient.java index cd4263ff9..aa1a6ffc0 100644 --- a/src/main/java/com/pinterest/secor/parser/QuboleClient.java +++ b/src/main/java/com/pinterest/secor/parser/QuboleClient.java @@ -31,6 +31,8 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class QuboleClient { + private static final long MAX_QUBOLE_WAIT_TIME_MS = 300000; // 5 minutes + private String mApiToken; public QuboleClient(SecorConfig config) { @@ -94,24 +96,28 @@ private int query(String query) throws IOException { return (Integer) response.get("id"); } - private void waitForCompletion(int commandId) throws IOException, InterruptedException { + private int waitForCompletion(int commandId, long timeout) throws IOException, InterruptedException { URL url = new URL("https://api.qubole.com/api/v1.2/commands/" + commandId); - while (true) { + long endTime = System.currentTimeMillis() + timeout; + + while (System.currentTimeMillis() < endTime) { Map response = makeRequest(url, null); if (response.get("status").equals("done")) { - return; + return 0; } System.out.println("waiting 3 seconds for results of query " + commandId + ". Current status " + response.get("status")); Thread.sleep(3000); } + + return 1; // qubole call failed to return within timeout } - public void addPartition(String table, String partition) throws IOException, + public int 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); + return waitForCompletion(commandId, MAX_QUBOLE_WAIT_TIME_MS); } } From 7a8f77df242218b74770c4ebd035d175c57ae446 Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Mon, 3 Oct 2016 16:32:08 -0700 Subject: [PATCH 202/330] instead of return, throw io exception, and add skip qubole config --- .../java/com/pinterest/secor/common/SecorConfig.java | 2 ++ .../pinterest/secor/parser/PartitionFinalizer.java | 2 +- .../com/pinterest/secor/parser/QuboleClient.java | 12 ++++++------ 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 171d6ce2e..29efee921 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -376,6 +376,8 @@ public String getHiveTableName(String topic) { return mProperties.getString(key, null); } + public boolean getSkipQuboleAddPartition() { return getBoolean("secor.skip.qubole.add.partition", false); } + public String getCompressionCodec() { return getString("secor.compression.codec"); } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 150b81f78..289f27066 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -168,7 +168,7 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw LOG.warn("HivePrefix is not defined. Skip hive registration"); } } - if (hiveTableName != null) { + if (hiveTableName != null && !mConfig.getSkipQuboleAddPartition()) { mQuboleClient.addPartition(hiveTableName, sb.toString()); } } catch (Exception e) { diff --git a/src/main/java/com/pinterest/secor/parser/QuboleClient.java b/src/main/java/com/pinterest/secor/parser/QuboleClient.java index aa1a6ffc0..70f2a8f71 100644 --- a/src/main/java/com/pinterest/secor/parser/QuboleClient.java +++ b/src/main/java/com/pinterest/secor/parser/QuboleClient.java @@ -96,28 +96,28 @@ private int query(String query) throws IOException { return (Integer) response.get("id"); } - private int waitForCompletion(int commandId, long timeout) throws IOException, InterruptedException { + 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 0; + return; } System.out.println("waiting 3 seconds for results of query " + commandId + ". Current status " + response.get("status")); Thread.sleep(3000); } - return 1; // qubole call failed to return within timeout + throw new IOException("Qubole commandId" + commandId + " failed to return within timeout."); } - public int addPartition(String table, String partition) throws IOException, + 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); - return waitForCompletion(commandId, MAX_QUBOLE_WAIT_TIME_MS); + waitForCompletion(commandId, MAX_QUBOLE_WAIT_TIME_MS); } } From 5d1a478418183f1811eee419f0015d0d29544867 Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Mon, 3 Oct 2016 17:47:00 -0700 Subject: [PATCH 203/330] add config for timeout and add default jvm configs to common properties --- src/main/config/secor.common.properties | 6 ++++++ .../java/com/pinterest/secor/common/SecorConfig.java | 12 +++++++++++- .../pinterest/secor/parser/PartitionFinalizer.java | 2 +- .../com/pinterest/secor/parser/QuboleClient.java | 3 ++- 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 7b1a210b5..bf1786a2f 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -302,3 +302,9 @@ 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 \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 29efee921..897d53913 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -376,7 +376,13 @@ public String getHiveTableName(String topic) { return mProperties.getString(key, null); } - public boolean getSkipQuboleAddPartition() { return getBoolean("secor.skip.qubole.add.partition", false); } + public boolean getQuboleEnabled() { + return getBoolean("secor.enable.qubole", true); + } + + public long getQuboleTimeoutMs() { + return getLong("secor.qubole.timeout.ms", 300000); + } public String getCompressionCodec() { return getString("secor.compression.codec"); @@ -482,6 +488,10 @@ public long getLong(String name) { return mProperties.getLong(name); } + public long getLong(String name, long defaultValue) { + return mProperties.getLong(name, defaultValue); + } + public String[] getStringArray(String name) { return mProperties.getStringArray(name); } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index 289f27066..bac754d78 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -168,7 +168,7 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw LOG.warn("HivePrefix is not defined. Skip hive registration"); } } - if (hiveTableName != null && !mConfig.getSkipQuboleAddPartition()) { + if (hiveTableName != null && mConfig.getQuboleEnabled()) { mQuboleClient.addPartition(hiveTableName, sb.toString()); } } catch (Exception e) { diff --git a/src/main/java/com/pinterest/secor/parser/QuboleClient.java b/src/main/java/com/pinterest/secor/parser/QuboleClient.java index 70f2a8f71..748ebc7d5 100644 --- a/src/main/java/com/pinterest/secor/parser/QuboleClient.java +++ b/src/main/java/com/pinterest/secor/parser/QuboleClient.java @@ -31,12 +31,13 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class QuboleClient { - private static final long MAX_QUBOLE_WAIT_TIME_MS = 300000; // 5 minutes + 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 { From 0fd9885f06d09f2152cca68e8021295425de62e2 Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Tue, 4 Oct 2016 12:07:58 -0700 Subject: [PATCH 204/330] remove method default values for secor convention --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index c7ccf15eb..65df442af 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -383,11 +383,11 @@ public String getHiveTableName(String topic) { } public boolean getQuboleEnabled() { - return getBoolean("secor.enable.qubole", true); + return getBoolean("secor.enable.qubole"); } public long getQuboleTimeoutMs() { - return getLong("secor.qubole.timeout.ms", 300000); + return getLong("secor.qubole.timeout.ms"); } public String getCompressionCodec() { From 6774d88cd67756a142ee5942578d8a1cb8b3951c Mon Sep 17 00:00:00 2001 From: Shawn Nguyen Date: Tue, 4 Oct 2016 13:53:20 -0700 Subject: [PATCH 205/330] remove unneeded getLong method with default value --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 65df442af..55ebc8af3 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -506,10 +506,6 @@ public long getLong(String name) { return mProperties.getLong(name); } - public long getLong(String name, long defaultValue) { - return mProperties.getLong(name, defaultValue); - } - public String[] getStringArray(String name) { return mProperties.getStringArray(name); } From 5d1c136e1436aeec8f909bc1b9fa87c562f799c2 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 4 Oct 2016 18:50:05 -0700 Subject: [PATCH 206/330] Bump up version to 0.23-SNAPSHOT and cut the version 0.22 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 351c35fd4..89a1ea891 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.22-SNAPSHOT + 0.23-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From dbd355e0b3c4a70c745c89e0c48ce2909078cea8 Mon Sep 17 00:00:00 2001 From: Dan Goldin Date: Sat, 8 Oct 2016 17:52:12 -0400 Subject: [PATCH 207/330] Support Kafka 0.10 and bump version --- pom.xml | 4 ++-- .../com/pinterest/secor/performance/PerformanceTest.java | 9 +++++++-- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 89a1ea891..2694d4132 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.23-SNAPSHOT + 0.24-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter @@ -109,7 +109,7 @@ org.apache.kafka kafka_2.10 - 0.8.2.1 + 0.10.0.1 org.slf4j diff --git a/src/test/java/com/pinterest/secor/performance/PerformanceTest.java b/src/test/java/com/pinterest/secor/performance/PerformanceTest.java index 59dd1e434..560c16f85 100644 --- a/src/test/java/com/pinterest/secor/performance/PerformanceTest.java +++ b/src/test/java/com/pinterest/secor/performance/PerformanceTest.java @@ -24,6 +24,8 @@ 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; @@ -48,6 +50,7 @@ import kafka.producer.KeyedMessage; import kafka.producer.ProducerConfig; import kafka.utils.ZKStringSerializer$; +import org.I0Itec.zkclient.ZkConnection; /** * A performance test for secor @@ -237,14 +240,16 @@ private static Map getTopicMetadata( 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(zkClient, topic, partitions, - replicationFactor, props); + AdminUtils.createTopic(zkUtils, topic, partitions, + replicationFactor, props, RackAwareMode.Disabled$.MODULE$); } } catch (TopicExistsException e) { System.out.println(e.getMessage()); From dce449bf46550134483c274045b21fcb250e4eef Mon Sep 17 00:00:00 2001 From: Prasanna Gautam Date: Sun, 9 Oct 2016 16:48:13 +0545 Subject: [PATCH 208/330] Support protobuf 3.1.0 and Google Timestamp type - Upgraded deps to protobuf 3.1.0 - Supports google.protobuf.Timestamp type for extracting times - Additional tests added as a copy of ProtobufMessageParserTest - proto2 files continue to work, but explicitly defined as such --- pom.xml | 12 ++- .../secor/parser/ProtobufMessageParser.java | 10 +- .../parser/TimestampedMessageParser.java | 6 ++ .../parser/ProtobufTimestampParserTest.java | 93 +++++++++++++++++++ src/test/protobuf/unittest.proto | 3 +- src/test/protobuf/unittesttimestamp.proto | 19 ++++ 6 files changed, 137 insertions(+), 6 deletions(-) create mode 100644 src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java create mode 100644 src/test/protobuf/unittesttimestamp.proto diff --git a/pom.xml b/pom.xml index 89a1ea891..669c9d520 100644 --- a/pom.xml +++ b/pom.xml @@ -89,7 +89,12 @@ com.google.protobuf protobuf-java - 2.6.1 + 3.1.0 + + + com.google.protobuf + protobuf-java-util + 3.1.0 com.amazonaws @@ -401,7 +406,7 @@ com.github.os72 protoc-jar-maven-plugin - 3.0.0-b3 + 3.1.0 protobuf-test-sources @@ -410,13 +415,14 @@ run - 2.6.1 + 3.1.0 src/test/protobuf src/test/protobuf + true java diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index afc2af7bf..1575a4676 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.regex.Pattern; +import com.google.protobuf.Descriptors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,8 +74,13 @@ public long extractTimestampMillis(final Message message) throws IOException { decodedMessage = (com.google.protobuf.Message) decodedMessage .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); } - return toMillis((Long) decodedMessage - .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i]))); + Object timestampObject = decodedMessage + .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); + if (timestampObject instanceof com.google.protobuf.Timestamp){ + return 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. diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 8a046f52f..2d767a861 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -16,6 +16,8 @@ */ package com.pinterest.secor.parser; +import com.google.protobuf.Timestamp; +import com.google.protobuf.util.Timestamps; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import org.slf4j.Logger; @@ -95,6 +97,10 @@ protected static long toMillis(final long timestamp) { return timestampMillis; } + protected static long toMillis(final Timestamp timestamp) { + return Timestamps.toMillis(timestamp); + } + public abstract long extractTimestampMillis(final Message message) throws Exception; protected String[] generatePartitions(long timestampMillis, boolean usingHourly, boolean usingMinutely) 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..dfbe825c6 --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java @@ -0,0 +1,93 @@ +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 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); + } + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + } + + @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 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()))); + } + + @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 = 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()))); + } +} diff --git a/src/test/protobuf/unittest.proto b/src/test/protobuf/unittest.proto index 6d975fadd..c06f8266d 100644 --- a/src/test/protobuf/unittest.proto +++ b/src/test/protobuf/unittest.proto @@ -1,5 +1,6 @@ -package com.pinterest.secor.protobuf; +syntax = "proto2"; +package com.pinterest.secor.protobuf; option java_package = "com.pinterest.secor.protobuf"; option java_outer_classname = "Messages"; 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 From 944d7c0dd3b1215cb73e60490cb0635c77f16a8e Mon Sep 17 00:00:00 2001 From: Prasanna Gautam Date: Mon, 10 Oct 2016 08:28:30 +0545 Subject: [PATCH 209/330] remove overloaded toMillis for protobuf use Timestamps.toMillis directly in ProtobufMessageParser --- .../com/pinterest/secor/parser/ProtobufMessageParser.java | 3 ++- .../com/pinterest/secor/parser/TimestampedMessageParser.java | 4 ---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index 1575a4676..1e5fb50be 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -20,6 +20,7 @@ import java.util.regex.Pattern; import com.google.protobuf.Descriptors; +import com.google.protobuf.util.Timestamps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,7 +78,7 @@ public long extractTimestampMillis(final Message message) throws IOException { Object timestampObject = decodedMessage .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); if (timestampObject instanceof com.google.protobuf.Timestamp){ - return toMillis((com.google.protobuf.Timestamp)timestampObject); + return Timestamps.toMillis((com.google.protobuf.Timestamp) timestampObject); }else { return toMillis((Long) timestampObject); } diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 2d767a861..5d5f2bcf2 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -97,10 +97,6 @@ protected static long toMillis(final long timestamp) { return timestampMillis; } - protected static long toMillis(final Timestamp timestamp) { - return Timestamps.toMillis(timestamp); - } - public abstract long extractTimestampMillis(final Message message) throws Exception; protected String[] generatePartitions(long timestampMillis, boolean usingHourly, boolean usingMinutely) From 896bf7f8a94b3bfd80ac7ce4654415eaab7a0409 Mon Sep 17 00:00:00 2001 From: Morri Feldman Date: Tue, 11 Oct 2016 16:16:35 +0300 Subject: [PATCH 210/330] Allow upload on oldest file per topic per partition Add boolean configuration secor.file.age.youngest --- src/main/config/secor.common.properties | 4 ++++ .../pinterest/secor/common/FileRegistry.java | 17 ++++++++++++++--- .../com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../secor/common/FileRegistryTest.java | 1 + 4 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 7b572d360..82bdcd9df 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -319,3 +319,7 @@ 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 diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index fe43a5151..eb67e0213 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -258,7 +258,12 @@ public long getModificationAgeSec(TopicPartition topicPartition) throws IOExcept public long getModificationAgeSec(TopicPartitionGroup topicPartitionGroup) throws IOException { long now = System.currentTimeMillis() / 1000L; - long result = Long.MAX_VALUE; + 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); @@ -267,8 +272,14 @@ public long getModificationAgeSec(TopicPartitionGroup topicPartitionGroup) throw 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) { diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 55ebc8af3..ae28465c7 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -143,6 +143,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"); } diff --git a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java index d7a08ce6f..a40844cff 100644 --- a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java +++ b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java @@ -57,6 +57,7 @@ public void setUp() throws Exception { 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); From 0033244f28b30a985a42f47e3eaa40dbd8225493 Mon Sep 17 00:00:00 2001 From: Ahsan Nabi Dar Date: Fri, 14 Oct 2016 14:21:49 +0800 Subject: [PATCH 211/330] Update secor.prod.properties --- src/main/config/secor.prod.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/config/secor.prod.properties b/src/main/config/secor.prod.properties index 5c119681b..f2806b2b9 100644 --- a/src/main/config/secor.prod.properties +++ b/src/main/config/secor.prod.properties @@ -23,7 +23,7 @@ include=secor.common.properties # 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 From 50fe0dee073634adaa572c0a314e5f6b86685f74 Mon Sep 17 00:00:00 2001 From: Prasanna Gautam Date: Mon, 17 Oct 2016 07:07:34 +0545 Subject: [PATCH 212/330] expose an extractTimestampMillis on just bytes For ProtobufMessageParser. The reason here is that while it subclasses from TimestampedMessageParser and as such would need to just expose on Message type by default, it's useful for verification to re-run these on just bytes too. This doesn't change the underlying behavior, other than adding an additional function call. --- .../pinterest/secor/parser/ProtobufMessageParser.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index 1e5fb50be..3ae8b9a16 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -67,9 +67,13 @@ public ProtobufMessageParser(SecorConfig config) { @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(message.getTopic(), - message.getPayload()); + com.google.protobuf.Message decodedMessage = protobufUtil.decodeMessage(topic, + bytes); int i = 0; for (; i < timestampFieldPath.length - 1; ++i) { decodedMessage = (com.google.protobuf.Message) decodedMessage @@ -86,7 +90,7 @@ public long extractTimestampMillis(final Message message) throws IOException { // Assume that the timestamp field is the first field, is required, // and is a uint64. - CodedInputStream input = CodedInputStream.newInstance(message.getPayload()); + CodedInputStream input = CodedInputStream.newInstance(bytes); // Don't really care about the tag, but need to read it to get, to // the payload. input.readTag(); From 8c8738f2e6a20ed582dd85630deb1f05c5361faa Mon Sep 17 00:00:00 2001 From: Ahsan Nabi Dar Date: Mon, 17 Oct 2016 10:49:18 +0800 Subject: [PATCH 213/330] Update README.md instruction on how to use gzip compression for file upload --- README.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/README.md b/README.md index f3e394503..9f72de898 100644 --- a/README.md +++ b/README.md @@ -74,6 +74,9 @@ Currently secor supports the following output formats - **[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.*=`. +- **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 implementing interactions with the environment. From c1e3410620ba36706a863e572c44065098a1bb0a Mon Sep 17 00:00:00 2001 From: Ahsan Nabi Dar Date: Mon, 17 Oct 2016 11:20:10 +0800 Subject: [PATCH 214/330] Update secor.common.properties --- src/main/config/secor.common.properties | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 82bdcd9df..670e6407b 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -106,6 +106,10 @@ secor.gs.bucket=secor_gs # Google cloud storage path where files are stored within the bucket. secor.gs.path=data +#partiotion uplaod in Google Clour Storage +secor.gs.path.partition.daily=false +secor.gs.path.partition.hourly=false + # Use direct uploads # WARNING: disables resumable uploads, files are uploaded in a single request # This may help prevent IOException: insufficient data written, From aee3713b62a000c58db70e939afb557d9b3634ef Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Mon, 17 Oct 2016 13:03:19 +0800 Subject: [PATCH 215/330] add folder structure for GCS, daily and hourly --- src/main/config/secor.common.properties | 32 ++++----- .../pinterest/secor/common/SecorConfig.java | 66 ++++++++++++------- .../secor/uploader/GsUploadManager.java | 23 ++++++- 3 files changed, 80 insertions(+), 41 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 670e6407b..0fc092036 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -108,7 +108,9 @@ secor.gs.path=data #partiotion uplaod in Google Clour Storage secor.gs.path.partition.daily=false +secor.gs.path.partition.daily.format=yyyyMMdd secor.gs.path.partition.hourly=false +secor.gs.path.partition.hourly=hh # Use direct uploads # WARNING: disables resumable uploads, files are uploaded in a single request @@ -132,9 +134,9 @@ secor.zookeeper.path=/ # Impacts how frequently the upload logic is triggered if no messages are delivered. kafka.consumer.timeout.ms=10000 -# 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. +# 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 @@ -197,8 +199,8 @@ secor.offsets.per.partition=10000000 secor.topic_partition.forget.seconds=600 # Setting the partitioner to use hourly partition -# By default, the partitioner will do daily partition, so the data will be -# written into +# 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 @@ -209,7 +211,7 @@ secor.topic_partition.forget.seconds=600 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 +# 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 @@ -224,15 +226,15 @@ 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 +# 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 +# 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 +# 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. @@ -252,7 +254,7 @@ statsd.hostport= # 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. +# 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=. @@ -275,8 +277,8 @@ message.timestamp.input.pattern= # 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 +# 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= @@ -299,7 +301,7 @@ 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. +# 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. diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index ae28465c7..abda8b01e 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -64,7 +64,7 @@ public static SecorConfig load() throws ConfigurationException { /** * Exposed for testability - * + * * @param properties */ public SecorConfig(PropertiesConfiguration properties) { @@ -160,7 +160,7 @@ public int getMessagesPerSecond() { public boolean getSeparateContainersForTopics() { return getString("secor.swift.containers.for.each.topic").toLowerCase().equals("true"); } - + public String getSwiftContainer() { return getString("secor.swift.container"); } @@ -168,7 +168,7 @@ public String getSwiftContainer() { public String getSwiftPath() { return getString("secor.swift.path"); } - + public String getS3Bucket() { return getString("secor.s3.bucket"); } @@ -232,7 +232,7 @@ public String getUploadManagerClass() { public String getMessageTransformerClass(){ return getString("secor.message.transformer.class"); } - + public int getTopicPartitionForgetSeconds() { return getInt("secor.topic_partition.forget.seconds"); } @@ -252,7 +252,7 @@ public int getOstrichPort() { public String getCloudService() { return getString("cloud.service"); } - + public String getAwsAccessKey() { return getString("aws.access.key"); } @@ -268,15 +268,15 @@ public String getAwsEndpoint() { public String getAwsRole() { return getString("aws.role"); } - + 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"); } @@ -300,35 +300,35 @@ public String getAwsSseCustomerKey() { 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"); } @@ -352,7 +352,7 @@ public String getMonitoringPrefix() { public String getMessageTimestampName() { return getString("message.timestamp.name"); } - + public String getMessageTimestampNameSeparator() { return getString("message.timestamp.name.separator"); } @@ -377,8 +377,8 @@ public int getFinalizerLookbackPeriods() { return getInt("secor.finalizer.lookback.periods", 10); } - public String getHivePrefix() { - return getString("secor.hive.prefix"); + public String getHivePrefix() { + return getString("secor.hive.prefix"); } public String getHiveTableName(String topic) { @@ -401,11 +401,11 @@ public String getCompressionCodec() { public int getMaxMessageSizeBytes() { return getInt("secor.max.message.size.bytes"); } - + public String getFileReaderWriterFactory() { return getString("secor.file.reader.writer.factory"); } - + public String getPerfTestTopicPrefix() { return getString("secor.kafka.perf_topic_prefix"); } @@ -426,6 +426,22 @@ public String getGsPath() { return getString("secor.gs.path"); } + public Boolean getGsPathPartitionHourly() { + return getBoolean("secor.gs.path.partition.hourly"); + } + + public String getGsPathPartitionHourlyFormat() { + return getString("secor.gs.path.partition.hourly.format"); + } + + public Boolean getGsPathPartitionDaily() { + return getBoolean("secor.gs.path.partition.daily"); + } + + public String getGsPathPartitionDailyFormat() { + return getString("secor.gs.path.partition.daily.format"); + } + public int getGsConnectTimeoutInMs() { return getInt("secor.gs.connect.timeout.ms", 3 * 60000); } @@ -455,7 +471,7 @@ public boolean getS3MD5HashPrefix() { 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); @@ -467,7 +483,7 @@ public Map getProtobufMessageClassPerTopic() { } return protobufClasses; } - + public TimeZone getTimeZone() { String timezone = getString("secor.parser.timezone"); return Strings.isNullOrEmpty(timezone) ? TimeZone.getTimeZone("UTC") : TimeZone.getTimeZone(timezone); @@ -476,7 +492,7 @@ public TimeZone getTimeZone() { public boolean getBoolean(String name, boolean defaultValue) { return mProperties.getBoolean(name, defaultValue); } - + public boolean getBoolean(String name) { return mProperties.getBoolean(name); } diff --git a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java index 26064444a..4419cdcf9 100644 --- a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java @@ -18,6 +18,7 @@ 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; @@ -29,6 +30,8 @@ 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. @@ -67,7 +70,8 @@ public GsUploadManager(SecorConfig config) throws Exception { @Override public Handle upload(LogFilePath localPath) throws Exception { final String gsBucket = mConfig.getGsBucket(); - final String gsKey = localPath.withPrefix(mConfig.getGsPath()).getLogFilePath(); + final String gsPath = gsPathPartitons(mConfig.getGsPath()); + final String gsKey = localPath.withPrefix(gsPath).getLogFilePath(); final File localFile = new File(localPath.getLogFilePath()); final boolean directUpload = mConfig.getGsDirectUpload(); @@ -150,4 +154,21 @@ public void initialize(HttpRequest httpRequest) throws IOException { }; } + private String gsPathPartitons(String gsPath) { + if (!mConfig.getGsPathPartitionDaily()){ + return gsPath; + } + Date dateCurrent = new Date(); + SimpleDateFormat ftDate = new SimpleDateFormat (mConfig.getGsPathPartitionDailyFormat()); + String datedPartition = ftDate.format(dateCurrent); + String gsPathPartitioned = StringUtils.join(new String [] {gsPath, datedPartition}, '/'); + + if (mConfig.getGsPathPartitionHourly()) { + SimpleDateFormat ftHour = new SimpleDateFormat (mConfig.getGsPathPartitionHourlyFormat()); + String hourlyPartition = ftHour.format(dateCurrent); + gsPathPartitioned = StringUtils.join(new String [] {gsPathPartitioned, hourlyPartition}, '/'); + } + return gsPathPartitioned; + } + } From 24619fcbc345fd93fcd096adce6075a588ad25a2 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Mon, 17 Oct 2016 18:36:03 +0800 Subject: [PATCH 216/330] fixed config --- src/main/config/secor.common.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 0fc092036..d74cadab8 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -110,7 +110,7 @@ secor.gs.path=data secor.gs.path.partition.daily=false secor.gs.path.partition.daily.format=yyyyMMdd secor.gs.path.partition.hourly=false -secor.gs.path.partition.hourly=hh +secor.gs.path.partition.hourly.format=hh # Use direct uploads # WARNING: disables resumable uploads, files are uploaded in a single request From e26b10ef49b7858219381b1da73cedfb5a646291 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Tue, 18 Oct 2016 09:36:22 +0800 Subject: [PATCH 217/330] config updated --- src/main/config/secor.dev.gs.properties | 6 ++++++ .../java/com/pinterest/secor/uploader/GsUploadManager.java | 4 ++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/main/config/secor.dev.gs.properties b/src/main/config/secor.dev.gs.properties index 5eb765bb1..f5ce1fe7c 100644 --- a/src/main/config/secor.dev.gs.properties +++ b/src/main/config/secor.dev.gs.properties @@ -13,6 +13,12 @@ secor.gs.bucket=secor_gs # Google cloud storage path where files are stored within the bucket. secor.gs.path=data +#partiotion uplaod in Google Clour Storage +secor.gs.path.partition.daily=false +secor.gs.path.partition.daily.format=yyyyMMdd +secor.gs.path.partition.hourly=false +secor.gs.path.partition.hourly.format=hh + ################ # END MUST SET # ################ diff --git a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java index 4419cdcf9..d9febd8a2 100644 --- a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java @@ -70,7 +70,7 @@ public GsUploadManager(SecorConfig config) throws Exception { @Override public Handle upload(LogFilePath localPath) throws Exception { final String gsBucket = mConfig.getGsBucket(); - final String gsPath = gsPathPartitons(mConfig.getGsPath()); + final String gsPath = gsPathWithPartitons(mConfig.getGsPath()); final String gsKey = localPath.withPrefix(gsPath).getLogFilePath(); final File localFile = new File(localPath.getLogFilePath()); final boolean directUpload = mConfig.getGsDirectUpload(); @@ -154,7 +154,7 @@ public void initialize(HttpRequest httpRequest) throws IOException { }; } - private String gsPathPartitons(String gsPath) { + private String gsPathWithPartitons(String gsPath) { if (!mConfig.getGsPathPartitionDaily()){ return gsPath; } From 289837d9357afac20925810036e2c58c398260af Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Thu, 20 Oct 2016 11:55:51 +0800 Subject: [PATCH 218/330] flexible timestamp format --- src/main/config/secor.common.properties | 20 ++++--- src/main/config/secor.dev.gs.properties | 6 --- .../pinterest/secor/common/SecorConfig.java | 16 ------ .../parser/TimestampedMessageParser.java | 54 +++++++++++++++++-- .../secor/uploader/GsUploadManager.java | 20 +------ 5 files changed, 63 insertions(+), 53 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index d74cadab8..16e8f6236 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -106,12 +106,6 @@ secor.gs.bucket=secor_gs # Google cloud storage path where files are stored within the bucket. secor.gs.path=data -#partiotion uplaod in Google Clour Storage -secor.gs.path.partition.daily=false -secor.gs.path.partition.daily.format=yyyyMMdd -secor.gs.path.partition.hourly=false -secor.gs.path.partition.hourly.format=hh - # Use direct uploads # WARNING: disables resumable uploads, files are uploaded in a single request # This may help prevent IOException: insufficient data written, @@ -205,8 +199,18 @@ secor.topic_partition.forget.seconds=600 # 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=true -# partitioner.granularity.minute=true +partitioner.granularity.daily=true +partitioner.granularity.hour=true +partitioner.granularity.minute=true + +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 diff --git a/src/main/config/secor.dev.gs.properties b/src/main/config/secor.dev.gs.properties index f5ce1fe7c..5eb765bb1 100644 --- a/src/main/config/secor.dev.gs.properties +++ b/src/main/config/secor.dev.gs.properties @@ -13,12 +13,6 @@ secor.gs.bucket=secor_gs # Google cloud storage path where files are stored within the bucket. secor.gs.path=data -#partiotion uplaod in Google Clour Storage -secor.gs.path.partition.daily=false -secor.gs.path.partition.daily.format=yyyyMMdd -secor.gs.path.partition.hourly=false -secor.gs.path.partition.hourly.format=hh - ################ # END MUST SET # ################ diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index abda8b01e..9fcc5b2b0 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -426,22 +426,6 @@ public String getGsPath() { return getString("secor.gs.path"); } - public Boolean getGsPathPartitionHourly() { - return getBoolean("secor.gs.path.partition.hourly"); - } - - public String getGsPathPartitionHourlyFormat() { - return getString("secor.gs.path.partition.hourly.format"); - } - - public Boolean getGsPathPartitionDaily() { - return getBoolean("secor.gs.path.partition.daily"); - } - - public String getGsPathPartitionDailyFormat() { - return getString("secor.gs.path.partition.daily.format"); - } - public int getGsConnectTimeoutInMs() { return getInt("secor.gs.connect.timeout.ms", 3 * 60000); } diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 5d5f2bcf2..de77ffc5f 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -47,20 +47,38 @@ public abstract class TimestampedMessageParser extends MessageParser implements private final SimpleDateFormat mDtHrMinFormatter; private final SimpleDateFormat mMinFormatter; + private final String mDtFormat; + private final String mHrFormat; + private final String mMinFormat; + + private final String mDtPrefix; + private final String mHrPrefix; + private final String mMinPrefix; + + private final boolean mUsingDaily; private final boolean mUsingHourly; private final boolean mUsingMinutely; public TimestampedMessageParser(SecorConfig config) { super(config); + mUsingDaily = usingDaily(config); mUsingHourly = usingHourly(config); mUsingMinutely = usingMinutely(config); + + mDtFormat = usingDateFormat(config); + mHrFormat = usingHourFormat(config); + mMinFormat = usingMinuteFormat(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("yyyy-MM-dd"); + mDtFormatter = new SimpleDateFormat(); mDtFormatter.setTimeZone(config.getTimeZone()); mHrFormatter = new SimpleDateFormat("HH"); @@ -75,6 +93,10 @@ public TimestampedMessageParser(SecorConfig config) { mMinFormatter.setTimeZone(config.getTimeZone()); } + static boolean usingDaily(SecorConfig config) { + return config.getBoolean("partitioner.granularity.daily", false); + } + static boolean usingHourly(SecorConfig config) { return config.getBoolean("partitioner.granularity.hour", false); } @@ -83,6 +105,30 @@ 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"); + } + + 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.min.prefix", "min="); + } + protected static long toMillis(final long timestamp) { final long nanosecondDivider = (long) Math.pow(10, 9 + 9); final long millisecondDivider = (long) Math.pow(10, 9 + 3); @@ -102,9 +148,9 @@ protected static long toMillis(final long timestamp) { protected String[] generatePartitions(long timestampMillis, boolean usingHourly, boolean usingMinutely) throws Exception { Date date = new Date(timestampMillis); - String dt = "dt=" + mDtFormatter.format(date); - String hr = "hr=" + mHrFormatter.format(date); - String min = "min=" + mMinFormatter.format(date); + 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) { diff --git a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java index d9febd8a2..b7e3a175d 100644 --- a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java @@ -70,8 +70,7 @@ public GsUploadManager(SecorConfig config) throws Exception { @Override public Handle upload(LogFilePath localPath) throws Exception { final String gsBucket = mConfig.getGsBucket(); - final String gsPath = gsPathWithPartitons(mConfig.getGsPath()); - final String gsKey = localPath.withPrefix(gsPath).getLogFilePath(); + final String gsKey = localPath.withPrefix(mConfig.getGsPath()).getLogFilePath(); final File localFile = new File(localPath.getLogFilePath()); final boolean directUpload = mConfig.getGsDirectUpload(); @@ -154,21 +153,4 @@ public void initialize(HttpRequest httpRequest) throws IOException { }; } - private String gsPathWithPartitons(String gsPath) { - if (!mConfig.getGsPathPartitionDaily()){ - return gsPath; - } - Date dateCurrent = new Date(); - SimpleDateFormat ftDate = new SimpleDateFormat (mConfig.getGsPathPartitionDailyFormat()); - String datedPartition = ftDate.format(dateCurrent); - String gsPathPartitioned = StringUtils.join(new String [] {gsPath, datedPartition}, '/'); - - if (mConfig.getGsPathPartitionHourly()) { - SimpleDateFormat ftHour = new SimpleDateFormat (mConfig.getGsPathPartitionHourlyFormat()); - String hourlyPartition = ftHour.format(dateCurrent); - gsPathPartitioned = StringUtils.join(new String [] {gsPathPartitioned, hourlyPartition}, '/'); - } - return gsPathPartitioned; - } - } From b09824876739b9b13804a43a4bacb098fbcd1098 Mon Sep 17 00:00:00 2001 From: Ahsan Nabi Dar Date: Thu, 20 Oct 2016 12:00:22 +0800 Subject: [PATCH 219/330] Update secor.common.properties --- src/main/config/secor.common.properties | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 16e8f6236..5b25d1a1c 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -199,9 +199,8 @@ secor.topic_partition.forget.seconds=600 # 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.daily=true -partitioner.granularity.hour=true -partitioner.granularity.minute=true +partitioner.granularity.hour=false +partitioner.granularity.minute=false partitioner.granularity.date.prefix=dt= partitioner.granularity.hour.prefix=hr= From a1d682fc3d9d1b640dac9f7fdf9bc0b7c8390695 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Thu, 20 Oct 2016 12:15:27 +0800 Subject: [PATCH 220/330] added mtDtFormat --- .../com/pinterest/secor/parser/TimestampedMessageParser.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index de77ffc5f..12b388734 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -70,6 +70,7 @@ public TimestampedMessageParser(SecorConfig config) { mHrFormat = usingHourFormat(config); mMinFormat = usingMinuteFormat(config); + mDtPrefix = usingDatePrefix(config); mHrPrefix = usingHourPrefix(config); mMinPrefix = usingMinutePrefix(config); From ea9f377f1817336bf17c43583450c59950b0e8ec Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Thu, 20 Oct 2016 16:26:06 +0800 Subject: [PATCH 221/330] updated tests --- .../parser/TimestampedMessageParser.java | 20 ++++++++----------- .../secor/parser/JsonMessageParserTest.java | 13 +++++++++--- .../secor/parser/MessagePackParserTest.java | 10 ++++++++++ .../parser/ProtobufMessageParserTest.java | 6 ++++++ .../parser/ProtobufTimestampParserTest.java | 6 ++++++ .../secor/parser/RegexMessageParserTest.java | 7 +++++++ .../secor/parser/ThriftMessageParserTest.java | 6 ++++++ 7 files changed, 53 insertions(+), 15 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 12b388734..48b40a571 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -55,17 +55,15 @@ public abstract class TimestampedMessageParser extends MessageParser implements private final String mHrPrefix; private final String mMinPrefix; - private final boolean mUsingDaily; private final boolean mUsingHourly; private final boolean mUsingMinutely; public TimestampedMessageParser(SecorConfig config) { super(config); - mUsingDaily = usingDaily(config); + mUsingHourly = usingHourly(config); mUsingMinutely = usingMinutely(config); - mDtFormat = usingDateFormat(config); mHrFormat = usingHourFormat(config); mMinFormat = usingMinuteFormat(config); @@ -79,23 +77,21 @@ public TimestampedMessageParser(SecorConfig config) { mFinalizerDelaySeconds = config.getFinalizerDelaySeconds(); LOG.info("FinalizerDelaySeconds: {}", mFinalizerDelaySeconds); - mDtFormatter = new SimpleDateFormat(); + mDtFormatter = new SimpleDateFormat(mDtFormat); mDtFormatter.setTimeZone(config.getTimeZone()); - mHrFormatter = new SimpleDateFormat("HH"); + mHrFormatter = new SimpleDateFormat(mHrFormat); mHrFormatter.setTimeZone(config.getTimeZone()); - mDtHrFormatter = new SimpleDateFormat("yyyy-MM-dd-HH"); + mMinFormatter = new SimpleDateFormat(mMinFormat); + mMinFormatter.setTimeZone(config.getTimeZone()); + + mDtHrFormatter = new SimpleDateFormat(mDtFormat+ "-" + mHrFormat); mDtHrFormatter.setTimeZone(config.getTimeZone()); - mDtHrMinFormatter = new SimpleDateFormat("yyyy-MM-dd-HH-mm"); + mDtHrMinFormatter = new SimpleDateFormat(mDtFormat+ "-" + mHrFormat + "-" + mMinFormat); mDtHrMinFormatter.setTimeZone(config.getTimeZone()); - mMinFormatter = new SimpleDateFormat("mm"); - mMinFormatter.setTimeZone(config.getTimeZone()); - } - static boolean usingDaily(SecorConfig config) { - return config.getBoolean("partitioner.granularity.daily", false); } static boolean usingHourly(SecorConfig config) { diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index 39c6c88b7..964287d99 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -46,6 +46,13 @@ public void setUp() throws Exception { 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\":\"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); @@ -61,7 +68,7 @@ public void setUp() throws Exception { 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, null, messageWithoutTimestamp); - + 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); @@ -79,12 +86,12 @@ public void testExtractTimestampMillis() throws Exception { assertEquals(0l, jsonMessageParser.extractTimestampMillis(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.extractTimestampMillis(mMessageWithNestedTimestamp)); } diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java index f21f56a92..5b7e98748 100644 --- a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -45,9 +45,17 @@ public void setUp() throws Exception { SecorConfig 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()); + HashMap mapWithSecondTimestamp = new HashMap(); mapWithSecondTimestamp.put("ts", 1405970352); mMessageWithSecondsTimestamp = new Message("test", 0, 0, null, @@ -78,6 +86,8 @@ public void setUp() throws Exception { mMessageWithMillisStringTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWithMillisStringTimestamp)); + + } @Test diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index 043bcdf70..7883a61ba 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -46,6 +46,12 @@ private Message buildMessage(long timestamp) throws Exception { @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="); } @Test diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java index dfbe825c6..c2c082861 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java @@ -33,6 +33,12 @@ private Message buildMessage(long timestamp) throws Exception { @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="); } @Test diff --git a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java index 13edce801..1c32af6cc 100644 --- a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -37,6 +37,13 @@ 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="); + 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); diff --git a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java index d7c9e2af6..b0a891cc7 100644 --- a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java @@ -19,6 +19,12 @@ public class ThriftMessageParserTest extends TestCase { @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="); } private Message buildMessage(long timestamp, int timestampTwo, long timestampThree) throws Exception { From 0c37f9bd6b333fd9628a3da5920d1777d02f2711 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Fri, 21 Oct 2016 10:56:23 +0800 Subject: [PATCH 222/330] DailyOffsetMessagePArser --- .../parser/DailyOffsetMessageParser.java | 69 +++++++++++++++++++ 1 file changed, 69 insertions(+) create mode 100644 src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java 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..e06907576 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java @@ -0,0 +1,69 @@ +/** + * 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 org.apache.commons.lang.StringUtils; + +import java.util.Date; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; + + + +/** + * Offset message parser groups messages based on the offset ranges. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class DailyOffsetMessageParser extends TimestampedMessageParser { + + private final boolean dUsingHourly; + private final boolean dUsingMinutely; + + public DailyOffsetMessageParser(SecorConfig config) { + super(config); + dUsingHourly = usingHourly(config); + dUsingMinutely = usingMinutely(config); + + } + + static boolean usingHourly(SecorConfig config) { + return config.getBoolean("partitioner.granularity.hour", false); + } + + static boolean usingMinutely(SecorConfig config) { + return config.getBoolean("partitioner.granularity.minute", false); + } + + @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(), dUsingHourly, dUsingMinutely); + String dailyPartitionPath = StringUtils.join(dailyPartition, '/'); + String[] result = {dailyPartitionPath, "offset=" + partition}; + return result; + } + + @Override + public long extractTimestampMillis(final Message message) { + return new Date().getTime(); //Daily Timestamp generation + } + +} From 859c9a5fb6f5bbbaec17d44b17f3d1c5a6aae7d1 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Sat, 22 Oct 2016 11:13:42 +0800 Subject: [PATCH 223/330] changed access modifier --- .../secor/parser/DailyOffsetMessageParser.java | 16 +--------------- .../secor/parser/TimestampedMessageParser.java | 4 ++-- 2 files changed, 3 insertions(+), 17 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java index e06907576..8ed92bf03 100644 --- a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java @@ -32,22 +32,8 @@ */ public class DailyOffsetMessageParser extends TimestampedMessageParser { - private final boolean dUsingHourly; - private final boolean dUsingMinutely; - public DailyOffsetMessageParser(SecorConfig config) { super(config); - dUsingHourly = usingHourly(config); - dUsingMinutely = usingMinutely(config); - - } - - static boolean usingHourly(SecorConfig config) { - return config.getBoolean("partitioner.granularity.hour", false); - } - - static boolean usingMinutely(SecorConfig config) { - return config.getBoolean("partitioner.granularity.minute", false); } @Override @@ -55,7 +41,7 @@ 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(), dUsingHourly, dUsingMinutely); + String[] dailyPartition = generatePartitions(new Date().getTime(), mUsingHourly, mUsingMinutely); String dailyPartitionPath = StringUtils.join(dailyPartition, '/'); String[] result = {dailyPartitionPath, "offset=" + partition}; return result; diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 48b40a571..cf992bcbe 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -55,8 +55,8 @@ public abstract class TimestampedMessageParser extends MessageParser implements private final String mHrPrefix; private final String mMinPrefix; - private final boolean mUsingHourly; - private final boolean mUsingMinutely; + protected final boolean mUsingHourly; + protected final boolean mUsingMinutely; public TimestampedMessageParser(SecorConfig config) { From 42f440fe9ad5be43f44f2433c36f8bf0973e29b3 Mon Sep 17 00:00:00 2001 From: Ahsan Nabi Dar Date: Mon, 24 Oct 2016 16:27:14 +0800 Subject: [PATCH 224/330] Update README.md --- README.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/README.md b/README.md index 9f72de898..c1db413a5 100644 --- a/README.md +++ b/README.md @@ -60,6 +60,16 @@ One of the convenience features of Secor is the ability to group messages and sa - **[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. +- **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. @@ -131,6 +141,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [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) ## Companies who use Secor @@ -146,6 +157,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Zalando](http://www.zalando.com) * [Rakuten](http://techblog.rakuten.co.jp/) * [Appsflyer](https://www.appsflyer.com) + * [Wego] (http://www.wego.com) ## Help From 896c958eff3f3a328f4282a649c05de2141bc6f9 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Thu, 27 Oct 2016 15:37:33 +0800 Subject: [PATCH 225/330] partition parser --- src/main/config/secor.common.properties | 2 +- .../parser/DailyOffsetMessageParser.java | 2 +- .../pinterest/secor/parser/MessageParser.java | 10 +++- .../secor/parser/OffsetMessageParser.java | 2 +- .../parser/PartitionedMessageParser.java | 53 +++++++++++++++++++ 5 files changed, 64 insertions(+), 5 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 5b25d1a1c..e89a8e75f 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -188,7 +188,7 @@ secor.messages.per.second=10000 # 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 diff --git a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java index 8ed92bf03..94d9e8a09 100644 --- a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java @@ -43,7 +43,7 @@ public String[] extractPartitions(Message message) throws Exception { long partition = (offset / offsetsPerPartition) * offsetsPerPartition; String[] dailyPartition = generatePartitions(new Date().getTime(), mUsingHourly, mUsingMinutely); String dailyPartitionPath = StringUtils.join(dailyPartition, '/'); - String[] result = {dailyPartitionPath, "offset=" + partition}; + String[] result = {dailyPartitionPath, offsetPrefix + partition}; return result; } diff --git a/src/main/java/com/pinterest/secor/parser/MessageParser.java b/src/main/java/com/pinterest/secor/parser/MessageParser.java index 16779950d..c2517df5e 100644 --- a/src/main/java/com/pinterest/secor/parser/MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/MessageParser.java @@ -37,10 +37,12 @@ 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 && @@ -50,6 +52,10 @@ public MessageParser(SecorConfig config) { } } + static String usingOffsetPrefix(SecorConfig config) { + return config.getString("secor.offsets.prefix", "offset="); + } + public ParsedMessage parse(Message message) throws Exception { String[] partitions = extractPartitions(message); return new ParsedMessage(message.getTopic(), message.getKafkaPartition(), @@ -58,11 +64,11 @@ public ParsedMessage parse(Message message) throws Exception { } public abstract String[] extractPartitions(Message payload) throws Exception; - + public Object getJsonFieldValue(JSONObject jsonObject) { Object fieldValue = null; if (mNestedFields != null) { - Object finalValue = 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()); 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/PartitionedMessageParser.java b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java new file mode 100644 index 000000000..e8b74442c --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.parser; + +import org.apache.commons.lang.StringUtils; + +import java.util.Date; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; + + + +/** + * Offset message parser groups messages based on the offset ranges. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class PartitionedMessageParser extends TimestampedMessageParser { + + public PartitionedMessageParser(SecorConfig config) { + super(config); + } + + @Override + public String[] extractPartitions(Message message) throws Exception { + long offset = message.getOffset(); + 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 + } + +} From 53adb7e81d068fd99c87a346e39b44b29a1593b5 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Thu, 27 Oct 2016 15:39:59 +0800 Subject: [PATCH 226/330] removed fetch offset --- .../com/pinterest/secor/parser/PartitionedMessageParser.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java index e8b74442c..c193faa28 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java @@ -38,7 +38,6 @@ public PartitionedMessageParser(SecorConfig config) { @Override public String[] extractPartitions(Message message) throws Exception { - long offset = message.getOffset(); String[] dailyPartition = generatePartitions(new Date().getTime(), mUsingHourly, mUsingMinutely); String dailyPartitionPath = StringUtils.join(dailyPartition, '/'); String[] result = {dailyPartitionPath}; From 014095cd9598d777683c1d78a514d83b39e998a4 Mon Sep 17 00:00:00 2001 From: Ahsan Nabi Dar Date: Fri, 28 Oct 2016 10:01:31 +0800 Subject: [PATCH 227/330] Update MessageParser.java removed default from java code --- src/main/java/com/pinterest/secor/parser/MessageParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/parser/MessageParser.java b/src/main/java/com/pinterest/secor/parser/MessageParser.java index c2517df5e..da10b7016 100644 --- a/src/main/java/com/pinterest/secor/parser/MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/MessageParser.java @@ -53,7 +53,7 @@ public MessageParser(SecorConfig config) { } static String usingOffsetPrefix(SecorConfig config) { - return config.getString("secor.offsets.prefix", "offset="); + return config.getString("secor.offsets.prefix"); } public ParsedMessage parse(Message message) throws Exception { From b4d25661d64c57b19fe1b356d625a1defb1d50a9 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Fri, 28 Oct 2016 12:22:57 +0800 Subject: [PATCH 228/330] updted tests --- .../java/com/pinterest/secor/util/ReflectionUtilTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java index 34ba65fa6..2cba294cc 100644 --- a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java +++ b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java @@ -33,8 +33,10 @@ 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"); + //Mockito.when(MessageParser.usingOffsetPrefix(mSecorConfig)).thenReturn("offset="); } @Test @@ -68,4 +70,4 @@ public void testFileWriterConstructorMissing() throws Exception { ReflectionUtil.createFileWriter("java.lang.Object", mLogFilePath, null, mSecorConfig); } -} \ No newline at end of file +} From 777f85501c8e5cba3475399b1a6edc344797ab92 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Fri, 28 Oct 2016 12:24:12 +0800 Subject: [PATCH 229/330] removed unnecesary addition --- src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java index 2cba294cc..37f1dcec4 100644 --- a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java +++ b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java @@ -36,7 +36,7 @@ public void setUp() throws Exception { properties.addProperty("secor.offsets.prefix","offset="); mSecorConfig = new SecorConfig(properties); mLogFilePath = new LogFilePath("/foo", "/foo/bar/baz/1_1_1"); - //Mockito.when(MessageParser.usingOffsetPrefix(mSecorConfig)).thenReturn("offset="); + } @Test From 840749e2d13e769ab32e8034c786d24a8c8e2a0a Mon Sep 17 00:00:00 2001 From: jsastre Date: Mon, 31 Oct 2016 17:53:35 +0100 Subject: [PATCH 230/330] parquet output for thrift messages support added --- README.md | 2 + pom.xml | 1041 +++++++++-------- .../pinterest/secor/common/SecorConfig.java | 32 + .../ThriftParquetFileReaderWriterFactory.java | 120 ++ .../secor/parser/ThriftMessageParser.java | 7 +- .../secor/tools/TestLogMessageProducer.java | 1 + .../com/pinterest/secor/util/ThriftUtil.java | 108 ++ ...iftParquetFileReaderWriterFactoryTest.java | 97 ++ .../secor/parser/ThriftMessageParserTest.java | 5 +- 9 files changed, 895 insertions(+), 518 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java create mode 100644 src/main/java/com/pinterest/secor/util/ThriftUtil.java create mode 100644 src/test/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactoryTest.java diff --git a/README.md b/README.md index c1db413a5..6069204e7 100644 --- a/README.md +++ b/README.md @@ -84,6 +84,8 @@ Currently secor supports the following output formats - **[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`. diff --git a/pom.xml b/pom.xml index 669c9d520..3e61012b2 100644 --- a/pom.xml +++ b/pom.xml @@ -1,407 +1,418 @@ - - 4.0.0 + + 4.0.0 - com.pinterest - secor - 0.23-SNAPSHOT - jar - secor - Kafka to s3/gs/swift logs exporter - https://github.com/pinterest/secor + com.pinterest + secor + 0.23-SNAPSHOT + jar + secor + Kafka to s3/gs/swift logs exporter + https://github.com/pinterest/secor - - - Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - repo - A business-friendly OSS license - - + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + A business-friendly OSS license + + - - - pgarbacki - Pawel Garbacki - - - yuyang - Yu Yang - - - ramki - Ramki Venkatachalam - - - hcai - Henry Cai - - + + + 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 - + + 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.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 - - + + + ossrh + https://oss.sonatype.org/service/local/staging/deploy/maven2/ + + + ossrh + https://oss.sonatype.org/content/repositories/snapshots + + - - - Twitter public Maven repo - http://maven.twttr.com - - - Typesafe public Maven repo - http://repo.typesafe.com/typesafe/releases - - - - - Twitter public Maven repo - http://maven.twttr.com - - - Typesafe public Maven repo - http://repo.typesafe.com/typesafe/releases - - + + + Twitter public Maven repo + http://maven.twttr.com + + + Typesafe public Maven repo + http://repo.typesafe.com/typesafe/releases + + + + + Twitter public Maven repo + http://maven.twttr.com + + + Typesafe public Maven repo + http://repo.typesafe.com/typesafe/releases + + - - - com.google.protobuf - protobuf-java + + + + com.google.protobuf + protobuf-java 3.1.0 com.google.protobuf protobuf-java-util 3.1.0 - - - com.amazonaws - aws-java-sdk-s3 - 1.10.68 - - - com.amazonaws - aws-java-sdk-sts - 1.10.68 - - - net.java.dev.jets3t - jets3t - 0.9.4 - - - org.apache.kafka - kafka_2.10 - 0.8.2.1 - - - org.slf4j - slf4j-simple - - - - - log4j - log4j - 1.2.15 - - - com.sun.jmx - jmxri - - - com.sun.jdmk - jmxtools - - - javax.jms - jms - - - - - org.slf4j - slf4j-api - 1.7.2 - - - org.slf4j - jcl-over-slf4j - 1.7.2 - - - org.slf4j - slf4j-log4j12 - 1.7.2 - - - commons-configuration - commons-configuration - 1.9 - + + + com.amazonaws + aws-java-sdk-s3 + 1.10.68 + + + com.amazonaws + aws-java-sdk-sts + 1.10.68 + + + net.java.dev.jets3t + jets3t + 0.9.4 + + + org.apache.kafka + kafka_2.10 + 0.8.2.1 + + + org.slf4j + slf4j-simple + + + + + log4j + log4j + 1.2.15 + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + javax.jms + jms + + + + + org.slf4j + slf4j-api + 1.7.2 + + + org.slf4j + jcl-over-slf4j + 1.7.2 + + + org.slf4j + slf4j-log4j12 + 1.7.2 + + + commons-configuration + commons-configuration + 1.9 + - - org.apache.hadoop - hadoop-common - 2.7.0 - - - org.apache.hadoop - 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.hadoop + hadoop-common + 2.7.0 + + + org.apache.hadoop + hadoop-mapreduce-client-core + 2.7.0 + + + org.apache.hadoop + 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 - 1.8.1 - - - org.apache.parquet - parquet-encoding - 1.8.1 - - - org.apache.parquet - parquet-column - 1.8.1 - - - org.apache.parquet - parquet-hadoop - 1.8.1 - - - org.apache.parquet - parquet-protobuf - 1.8.1 - + + + 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.thrift - libthrift - 0.5.0 - - - com.twitter - ostrich_2.10 - 9.2.1 - - - com.twitter.common.zookeeper - lock - 0.0.7 - - - com.google.guava - guava - 18.0 - - - net.minidev - json-smart - 2.0-RC3 - - - junit - junit - 4.11 - test - - - org.mockito - mockito-all - 1.9.5 - - - org.powermock - powermock-module-junit4 - 1.5.2 - test - - - org.powermock - powermock-api-mockito - 1.5.2 - test - - - 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.thrift + libthrift + 0.9.3 + + + com.twitter + ostrich_2.10 + 9.2.1 + + + com.twitter.common.zookeeper + lock + 0.0.7 + + + com.google.guava + guava + 18.0 + + + net.minidev + json-smart + 2.0-RC3 + + + junit + junit + 4.11 + test + + + org.mockito + mockito-all + 1.9.5 + + + org.powermock + powermock-module-junit4 + 1.5.2 + test + + + org.powermock + powermock-api-mockito + 1.5.2 + test + + + 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.parquet + parquet-thrift + ${parquet.version} + + - - - - src/main/config - - - - - src/test/config - - - - - org.codehaus.mojo - exec-maven-plugin - 1.1 - - - compile - - exec - - - git - - log - --pretty=format:build_revision=%H - -n1 - - target/build.properties - - - - - - - maven-dependency-plugin - - - package - - copy-dependencies - - - ${project.build.directory}/lib - - - - - - org.codehaus.mojo - exec-maven-plugin - 1.1 - - - compile - - exec - - - git - - log - --pretty=format:build_revision=%H - -n1 - - target/classes/com/pinterest/secor/common/build.properties - - - - - - org.apache.maven.plugins - maven-assembly-plugin - - - create-distribution - package - - single - - - - src/main/assembly/secor.xml - - - - - + + + + src/main/config + + + + + src/test/config + + + + + org.codehaus.mojo + exec-maven-plugin + 1.1 + + + compile + + exec + + + git + + log + --pretty=format:build_revision=%H + -n1 + + target/build.properties + + + + + + + maven-dependency-plugin + + + package + + copy-dependencies + + + ${project.build.directory}/lib + + + + + + org.codehaus.mojo + exec-maven-plugin + 1.1 + + + compile + + exec + + + git + + log + --pretty=format:build_revision=%H + -n1 + + target/classes/com/pinterest/secor/common/build.properties + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + create-distribution + package + + single + + + + src/main/assembly/secor.xml + + + + + com.github.os72 @@ -453,135 +464,135 @@ - - - com.twitter - maven-finagle-thrift-plugin - 0.0.9 - - - thrift-sources - generate-sources - - compile - - - - finagle - - - finagle - - - - - thrift-test-sources - generate-test-sources - - testCompile - - - - finagle - - - finagle - - - - - - - finagle - - - - - org.codehaus.mojo - license-maven-plugin - 1.6 - - - download-licenses - - download-licenses - - - - - - 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 - - - - + + + com.twitter + maven-finagle-thrift-plugin + 0.0.9 + + + thrift-sources + generate-sources + + compile + + + + finagle + + + finagle + + + + + thrift-test-sources + generate-test-sources + + testCompile + + + + finagle + + + finagle + + + + + + + finagle + + + + + org.codehaus.mojo + license-maven-plugin + 1.6 + + + download-licenses + + download-licenses + + + + + + 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 + + + + - - - 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 - - - - - - - - + + + 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 + + + + + + + + diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 9fcc5b2b0..c06d26778 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -426,6 +426,22 @@ public String getGsPath() { return getString("secor.gs.path"); } + public Boolean getGsPathPartitionHourly() { + return getBoolean("secor.gs.path.partition.hourly"); + } + + public String getGsPathPartitionHourlyFormat() { + return getString("secor.gs.path.partition.hourly.format"); + } + + public Boolean getGsPathPartitionDaily() { + return getBoolean("secor.gs.path.partition.daily"); + } + + public String getGsPathPartitionDailyFormat() { + return getString("secor.gs.path.partition.daily.format"); + } + public int getGsConnectTimeoutInMs() { return getInt("secor.gs.connect.timeout.ms", 3 * 60000); } @@ -468,6 +484,18 @@ public Map getProtobufMessageClassPerTopic() { 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); @@ -513,4 +541,8 @@ public long getLong(String name) { public String[] getStringArray(String name) { return mProperties.getStringArray(name); } + + public String getThriftProtocolClass() { + return mProperties.getString("secor.thrift.protocol.class"); + } } 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..2133c4761 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java @@ -0,0 +1,120 @@ +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.ThriftUtil; + +/** + * Adapted from com.pinterest.secor.io.impl.ProtobufParquetFileReaderWriterFactory + * Implementation for reading/writing thrift messages to/from Parquet files. + * + * @author jaime sastre (jaime.sastre.s@gmail.com) + */ +public class ThriftParquetFileReaderWriterFactory implements FileReaderWriterFactory { + + private ThriftUtil thriftUtil; + + public ThriftParquetFileReaderWriterFactory(SecorConfig config) { + thriftUtil = new ThriftUtil(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); + } + + @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/parser/ThriftMessageParser.java b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java index b9c7b810c..f1ef86d46 100644 --- a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java @@ -21,6 +21,7 @@ import org.apache.thrift.TDeserializer; import org.apache.thrift.TException; import org.apache.thrift.TFieldIdEnum; +import org.apache.thrift.protocol.TProtocolFactory; /** * Thrift message parser extracts date partitions from thrift messages. @@ -52,9 +53,11 @@ public String getFieldName() { } } - public ThriftMessageParser(SecorConfig config) { + public ThriftMessageParser(SecorConfig config) throws InstantiationException, IllegalAccessException, ClassNotFoundException { super(config); - mDeserializer = new TDeserializer(); + String factoryClassName = mConfig.getThriftProtocolClass().concat("$Factory"); + TProtocolFactory protocolFactory = ((Class) Class.forName(factoryClassName)).newInstance(); + mDeserializer = new TDeserializer(protocolFactory); mThriftPath = new ThriftPath(mConfig.getMessageTimestampName(), (short) mConfig.getMessageTimestampId()); mTimestampType = mConfig.getMessageTimestampType(); } diff --git a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java index a8eb86023..31b65433b 100644 --- a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java +++ b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java @@ -17,6 +17,7 @@ package com.pinterest.secor.tools; import kafka.javaapi.producer.Producer; + import kafka.producer.KeyedMessage; import kafka.producer.ProducerConfig; 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..fca96fdae --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/ThriftUtil.java @@ -0,0 +1,108 @@ +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.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 Class messageProtocolClass; + + /** + * 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 { + messageProtocolClass = ((Class) Class.forName(config.getThriftProtocolClass().concat("$Factory"))); + } catch (ClassNotFoundException 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(messageProtocolClass.newInstance()); + 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(messageProtocolClass.newInstance()); + return serializer.serialize(object); + } +} 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..6f854b9ef --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactoryTest.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.io.impl; + +import static org.junit.Assert.assertArrayEquals; + +import java.util.HashMap; +import java.util.Map; + +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.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()); + + 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/parser/ThriftMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java index b0a891cc7..543167910 100644 --- a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java @@ -40,6 +40,7 @@ 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); @@ -52,6 +53,7 @@ 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); @@ -64,7 +66,8 @@ 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))); From 27872a6faf6943c178e95ad4338374d175fc63d3 Mon Sep 17 00:00:00 2001 From: jaimess Date: Wed, 2 Nov 2016 10:56:21 +0100 Subject: [PATCH 231/330] documentations for new config added TBinaryProtocol by default for thirift uses. indentation fixed --- pom.xml | 1053 ++++++++--------- src/main/config/secor.common.properties | 8 + .../pinterest/secor/common/SecorConfig.java | 18 +- .../ThriftParquetFileReaderWriterFactory.java | 50 +- .../secor/parser/ThriftMessageParser.java | 24 +- .../secor/tools/TestLogMessageProducer.java | 1 - .../com/pinterest/secor/util/ThriftUtil.java | 64 +- 7 files changed, 617 insertions(+), 601 deletions(-) diff --git a/pom.xml b/pom.xml index 3e61012b2..3659fd304 100644 --- a/pom.xml +++ b/pom.xml @@ -1,418 +1,417 @@ - - 4.0.0 + + 4.0.0 - com.pinterest - secor - 0.23-SNAPSHOT - jar - secor - Kafka to s3/gs/swift logs exporter - https://github.com/pinterest/secor + com.pinterest + secor + 0.23-SNAPSHOT + jar + secor + Kafka to s3/gs/swift logs exporter + https://github.com/pinterest/secor - - - Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - repo - A business-friendly OSS license - - + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + A business-friendly OSS license + + - - - pgarbacki - Pawel Garbacki - - - yuyang - Yu Yang - - - ramki - Ramki Venkatachalam - - - hcai - Henry Cai - - + + + 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 - + + 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 - + + 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 - - + + + ossrh + https://oss.sonatype.org/service/local/staging/deploy/maven2/ + + + ossrh + https://oss.sonatype.org/content/repositories/snapshots + + - - - Twitter public Maven repo - http://maven.twttr.com - - - Typesafe public Maven repo - http://repo.typesafe.com/typesafe/releases - - - - - Twitter public Maven repo - http://maven.twttr.com - - - Typesafe public Maven repo - http://repo.typesafe.com/typesafe/releases - - + + + Twitter public Maven repo + http://maven.twttr.com + + + Typesafe public Maven repo + http://repo.typesafe.com/typesafe/releases + + + + + Twitter public Maven repo + http://maven.twttr.com + + + Typesafe public Maven repo + http://repo.typesafe.com/typesafe/releases + + - - - - com.google.protobuf - protobuf-java + + + com.google.protobuf + protobuf-java 3.1.0 com.google.protobuf protobuf-java-util 3.1.0 - - - com.amazonaws - aws-java-sdk-s3 - 1.10.68 - - - com.amazonaws - aws-java-sdk-sts - 1.10.68 - - - net.java.dev.jets3t - jets3t - 0.9.4 - - - org.apache.kafka - kafka_2.10 - 0.8.2.1 - - - org.slf4j - slf4j-simple - - - - - log4j - log4j - 1.2.15 - - - com.sun.jmx - jmxri - - - com.sun.jdmk - jmxtools - - - javax.jms - jms - - - - - org.slf4j - slf4j-api - 1.7.2 - - - org.slf4j - jcl-over-slf4j - 1.7.2 - - - org.slf4j - slf4j-log4j12 - 1.7.2 - - - commons-configuration - commons-configuration - 1.9 - - - - org.apache.hadoop - hadoop-common - 2.7.0 - - - org.apache.hadoop - hadoop-mapreduce-client-core - 2.7.0 - - - org.apache.hadoop - 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 - + + + com.amazonaws + aws-java-sdk-s3 + 1.10.68 + + + com.amazonaws + aws-java-sdk-sts + 1.10.68 + + + net.java.dev.jets3t + jets3t + 0.9.4 + + + org.apache.kafka + kafka_2.10 + 0.8.2.1 + + + org.slf4j + slf4j-simple + + + + + log4j + log4j + 1.2.15 + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + javax.jms + jms + + + + + org.slf4j + slf4j-api + 1.7.2 + + + org.slf4j + jcl-over-slf4j + 1.7.2 + + + org.slf4j + slf4j-log4j12 + 1.7.2 + + + commons-configuration + commons-configuration + 1.9 + - - - org.apache.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.hadoop + hadoop-common + 2.7.0 + + + org.apache.hadoop + hadoop-mapreduce-client-core + 2.7.0 + + + org.apache.hadoop + 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.thrift - libthrift - 0.9.3 - - - com.twitter - ostrich_2.10 - 9.2.1 - - - com.twitter.common.zookeeper - lock - 0.0.7 - - - com.google.guava - guava - 18.0 - - - net.minidev - json-smart - 2.0-RC3 - - - junit - junit - 4.11 - test - - - org.mockito - mockito-all - 1.9.5 - - - org.powermock - powermock-module-junit4 - 1.5.2 - test - - - org.powermock - powermock-api-mockito - 1.5.2 - test - - - 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.parquet - parquet-thrift - ${parquet.version} - - + + + 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 + libthrift + 0.9.3 + + + com.twitter + ostrich_2.10 + 9.2.1 + + + com.twitter.common.zookeeper + lock + 0.0.7 + + + com.google.guava + guava + 18.0 + + + net.minidev + json-smart + 2.0-RC3 + + + junit + junit + 4.11 + test + + + org.mockito + mockito-all + 1.9.5 + + + org.powermock + powermock-module-junit4 + 1.5.2 + test + + + org.powermock + powermock-api-mockito + 1.5.2 + test + + + 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 + + - - - - src/main/config - - - - - src/test/config - - - - - org.codehaus.mojo - exec-maven-plugin - 1.1 - - - compile - - exec - - - git - - log - --pretty=format:build_revision=%H - -n1 - - target/build.properties - - - - - - - maven-dependency-plugin - - - package - - copy-dependencies - - - ${project.build.directory}/lib - - - - - - org.codehaus.mojo - exec-maven-plugin - 1.1 - - - compile - - exec - - - git - - log - --pretty=format:build_revision=%H - -n1 - - target/classes/com/pinterest/secor/common/build.properties - - - - - - org.apache.maven.plugins - maven-assembly-plugin - - - create-distribution - package - - single - - - - src/main/assembly/secor.xml - - - - - + + + + src/main/config + + + + + src/test/config + + + + + org.codehaus.mojo + exec-maven-plugin + 1.1 + + + compile + + exec + + + git + + log + --pretty=format:build_revision=%H + -n1 + + target/build.properties + + + + + + + maven-dependency-plugin + + + package + + copy-dependencies + + + ${project.build.directory}/lib + + + + + + org.codehaus.mojo + exec-maven-plugin + 1.1 + + + compile + + exec + + + git + + log + --pretty=format:build_revision=%H + -n1 + + target/classes/com/pinterest/secor/common/build.properties + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + create-distribution + package + + single + + + + src/main/assembly/secor.xml + + + + + com.github.os72 @@ -464,135 +463,135 @@ - - - com.twitter - maven-finagle-thrift-plugin - 0.0.9 - - - thrift-sources - generate-sources - - compile - - - - finagle - - - finagle - - - - - thrift-test-sources - generate-test-sources - - testCompile - - - - finagle - - - finagle - - - - - - - finagle - - - - - org.codehaus.mojo - license-maven-plugin - 1.6 - - - download-licenses - - download-licenses - - - - - - 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 - - - - + + + com.twitter + maven-finagle-thrift-plugin + 0.0.9 + + + thrift-sources + generate-sources + + compile + + + + finagle + + + finagle + + + + + thrift-test-sources + generate-test-sources + + testCompile + + + + finagle + + + finagle + + + + + + + finagle + + + + + org.codehaus.mojo + license-maven-plugin + 1.6 + + + download-licenses + + download-licenses + + + + + + 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 + + + + - - - 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 - - - - - - - - + + + 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 + + + + + + + + diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index e89a8e75f..0ed133e87 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -254,6 +254,14 @@ monitoring.prefix=secor # 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.*= + # Name of field that contains timestamp for JSON, MessagePack, or Thrift message parser. (1405970352123) message.timestamp.name=timestamp diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index c06d26778..877030bd6 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -426,22 +426,6 @@ public String getGsPath() { return getString("secor.gs.path"); } - public Boolean getGsPathPartitionHourly() { - return getBoolean("secor.gs.path.partition.hourly"); - } - - public String getGsPathPartitionHourlyFormat() { - return getString("secor.gs.path.partition.hourly.format"); - } - - public Boolean getGsPathPartitionDaily() { - return getBoolean("secor.gs.path.partition.daily"); - } - - public String getGsPathPartitionDailyFormat() { - return getString("secor.gs.path.partition.daily.format"); - } - public int getGsConnectTimeoutInMs() { return getInt("secor.gs.connect.timeout.ms", 3 * 60000); } @@ -542,7 +526,7 @@ public String[] getStringArray(String name) { return mProperties.getStringArray(name); } - public String getThriftProtocolClass() { + public String getThriftProtocolClass() { return mProperties.getString("secor.thrift.protocol.class"); } } diff --git a/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java index 2133c4761..ca9ed9d4c 100644 --- a/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java @@ -20,7 +20,8 @@ import com.pinterest.secor.util.ThriftUtil; /** - * Adapted from com.pinterest.secor.io.impl.ProtobufParquetFileReaderWriterFactory + * Adapted from + * com.pinterest.secor.io.impl.ProtobufParquetFileReaderWriterFactory * Implementation for reading/writing thrift messages to/from Parquet files. * * @author jaime sastre (jaime.sastre.s@gmail.com) @@ -56,20 +57,20 @@ public ThriftParquetFileReader(LogFilePath logFilePath, CompressionCodec codec) } @SuppressWarnings("rawtypes") - @Override + @Override public KeyValue next() throws IOException { - TBase msg = reader.read(); - + 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 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; } @@ -82,15 +83,14 @@ public void close() throws IOException { protected class ThriftParquetFileWriter implements FileWriter { - @SuppressWarnings("rawtypes") - private ThriftParquetWriter writer; + @SuppressWarnings("rawtypes") + private ThriftParquetWriter writer; private String topic; - + @SuppressWarnings({ "rawtypes", "unchecked" }) - public ThriftParquetFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + public ThriftParquetFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { Path path = new Path(logFilePath.getLogFilePath()); - CompressionCodecName codecName = CompressionCodecName - .fromCompressionCodec(codec != null ? codec.getClass() : null); + CompressionCodecName codecName = CompressionCodecName.fromCompressionCodec(codec != null ? codec.getClass() : null); topic = logFilePath.getTopic(); writer = new ThriftParquetWriter(path, thriftUtil.getMessageClass(topic), codecName); } @@ -101,15 +101,15 @@ public long getLength() throws IOException { } @SuppressWarnings("unchecked") - @Override + @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); - } + try { + message = thriftUtil.decodeMessage(topic, keyValue.getValue()); + writer.write(message); + } catch (Exception e) { + throw new IOException("cannot write message", e); + } } @Override diff --git a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java index f1ef86d46..f3349c9b1 100644 --- a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java @@ -16,13 +16,16 @@ */ 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. * @@ -53,12 +56,21 @@ public String getFieldName() { } } - public ThriftMessageParser(SecorConfig config) throws InstantiationException, IllegalAccessException, ClassNotFoundException { + public ThriftMessageParser(SecorConfig config) + throws InstantiationException, IllegalAccessException, + ClassNotFoundException { super(config); - String factoryClassName = mConfig.getThriftProtocolClass().concat("$Factory"); - TProtocolFactory protocolFactory = ((Class) Class.forName(factoryClassName)).newInstance(); + 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()); + mThriftPath = new ThriftPath(mConfig.getMessageTimestampName(),(short) mConfig.getMessageTimestampId()); mTimestampType = mConfig.getMessageTimestampType(); } diff --git a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java index 31b65433b..a8eb86023 100644 --- a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java +++ b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java @@ -17,7 +17,6 @@ package com.pinterest.secor.tools; import kafka.javaapi.producer.Producer; - import kafka.producer.KeyedMessage; import kafka.producer.ProducerConfig; diff --git a/src/main/java/com/pinterest/secor/util/ThriftUtil.java b/src/main/java/com/pinterest/secor/util/ThriftUtil.java index fca96fdae..75a1d1a53 100644 --- a/src/main/java/com/pinterest/secor/util/ThriftUtil.java +++ b/src/main/java/com/pinterest/secor/util/ThriftUtil.java @@ -8,6 +8,7 @@ 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; @@ -15,10 +16,9 @@ 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: + * 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> * @@ -35,10 +35,10 @@ public class ThriftUtil { private boolean allTopics; @SuppressWarnings("rawtypes") - private Map> messageClassByTopic = new HashMap>(); + private Map> messageClassByTopic = new HashMap>(); @SuppressWarnings("rawtypes") - private Class messageClassForAll; - private Class messageProtocolClass; + private Class messageClassForAll; + private TProtocolFactory messageProtocolFactory; /** * Creates new instance of {@link ThriftUtil} @@ -49,13 +49,14 @@ public class ThriftUtil { * when configuration option * secor.thrift.message.class is invalid. */ - @SuppressWarnings({"rawtypes","unchecked"}) + @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()); + Class messageClass = (Class) Class.forName(entry.getValue()); allTopics = "*".equals(topic); @@ -68,14 +69,25 @@ public ThriftUtil(SecorConfig config) { } } catch (ClassNotFoundException e) { LOG.error("Unable to load thrift message class", e); - } + } } - + try { - messageProtocolClass = ((Class) Class.forName(config.getThriftProtocolClass().concat("$Factory"))); - } catch (ClassNotFoundException e) { - LOG.error("Unable to load thrift protocol class", e); - } + 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); + } } /** @@ -88,21 +100,23 @@ public ThriftUtil(SecorConfig config) { * configuration. */ @SuppressWarnings("rawtypes") - public Class getMessageClass(String topic) { + 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(messageProtocolClass.newInstance()); - TBase result = this.getMessageClass(topic).newInstance(); - serializer.deserialize(result, payload); - return result; + 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(messageProtocolClass.newInstance()); - return serializer.serialize(object); + public byte[] encodeMessage(TBase object) throws InstantiationException, + IllegalAccessException, TException { + TSerializer serializer = new TSerializer(messageProtocolFactory); + return serializer.serialize(object); } } From 18f02e7d33b8f09cb92fb9529310a764891c344d Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Thu, 3 Nov 2016 05:26:33 -0500 Subject: [PATCH 232/330] Extend ProgressMonitorMain to run in a loop. The interval between stats exports is set by `monitoring.interval.seconds`. Setting to zero (the default) causes the monitor to export once and then exit (existing behavior). --- README.md | 3 ++ src/main/config/secor.common.properties | 6 +++- .../pinterest/secor/common/SecorConfig.java | 8 ++++++ .../secor/main/ProgressMonitorMain.java | 28 ++++++++++++++++++- 4 files changed, 43 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index c1db413a5..a3b5cf11c 100644 --- a/README.md +++ b/README.md @@ -118,6 +118,9 @@ Progress monitor exports offset consumption lags per topic partition to [OpenTSD 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). diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index e89a8e75f..1fc92f5ee 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -247,9 +247,13 @@ tsdb.hostport= # Regex of topics that are not exported to TSDB. monitoring.blacklist.topics= -# Prefix of exported statss. +# 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 + # 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= diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 9fcc5b2b0..dd611ddea 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -349,6 +349,10 @@ public String getMonitoringPrefix() { return getString("monitoring.prefix"); } + public long getMonitoringIntervalSeconds() { + return getLong("monitoring.interval.seconds", 0); + } + public String getMessageTimestampName() { return getString("message.timestamp.name"); } @@ -510,6 +514,10 @@ public long getLong(String name) { return mProperties.getLong(name); } + public long getLong(String name, long defaultValue) { + return mProperties.getLong(name, defaultValue); + } + public String[] getStringArray(String name) { return mProperties.getStringArray(name); } 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); From f984e7b706053a94d2c0b8edf3aa935aa5fbe46c Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Thu, 3 Nov 2016 05:29:24 -0500 Subject: [PATCH 233/330] Make consumer group prefix on statsd metrics optional. The default is the current behavior, which is to include the consumer group as the overall prefix. Disable by setting `secor.prefixWithConsumerGroup` to false. --- src/main/config/secor.common.properties | 16 +++++++++ .../pinterest/secor/common/SecorConfig.java | 4 +++ .../secor/tools/ProgressMonitor.java | 34 ++++++++++++------- 3 files changed, 41 insertions(+), 13 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index e89a8e75f..0e61df0e5 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -254,6 +254,22 @@ monitoring.prefix=secor # Leave empty to disable this functionality. statsd.hostport= +# 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 9fcc5b2b0..0559c2675 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -341,6 +341,10 @@ public String getStatsDHostPort() { return getString("statsd.hostport"); } + public boolean getStatsDPrefixWithConsumerGroup(){ + return getBoolean("statsd.prefixWithConsumerGroup", true); + } + public String getMonitoringBlacklistTopics() { return getString("monitoring.blacklist.topics"); } diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 91e4c80b9..f9f06c140 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -63,6 +63,7 @@ public class ProgressMonitor { private KafkaClient mKafkaClient; private MessageParser mMessageParser; private String mPrefix; + private NonBlockingStatsDClient mStatsDClient; public ProgressMonitor(SecorConfig config) throws Exception @@ -77,6 +78,11 @@ public ProgressMonitor(SecorConfig config) 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 { @@ -139,7 +145,7 @@ public void exportStats() throws Exception { } // if there is a valid statsD port configured export to statsD - if (mConfig.getStatsDHostPort() != null && !mConfig.getStatsDHostPort().isEmpty()) { + if (mStatsDClient != null) { exportToStatsD(stats); } } @@ -148,22 +154,24 @@ public void exportStats() throws Exception { * Helper to publish stats to statsD client */ private void exportToStatsD(List stats) { - HostAndPort hostPort = HostAndPort.fromString(mConfig.getStatsDHostPort()); - // group stats by kafka group - NonBlockingStatsDClient client = new NonBlockingStatsDClient(mConfig.getKafkaGroup(), - hostPort.getHostText(), hostPort.getPort()); - for (Stat stat : stats) { @SuppressWarnings("unchecked") Map tags = (Map) stat.get(Stat.STAT_KEYS.TAGS.getName()); - String aspect = new StringBuilder((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(); - client.recordGaugeValue(aspect, Long.parseLong((String)stat.get(Stat.STAT_KEYS.VALUE.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); } } From d732eee4e0835e14e75578d10de44267a5b13aed Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Thu, 3 Nov 2016 06:00:56 -0500 Subject: [PATCH 234/330] fixup! Extend ProgressMonitorMain to run in a loop. --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index dd611ddea..7a3523bb2 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -350,7 +350,7 @@ public String getMonitoringPrefix() { } public long getMonitoringIntervalSeconds() { - return getLong("monitoring.interval.seconds", 0); + return getLong("monitoring.interval.seconds"); } public String getMessageTimestampName() { @@ -514,10 +514,6 @@ public long getLong(String name) { return mProperties.getLong(name); } - public long getLong(String name, long defaultValue) { - return mProperties.getLong(name, defaultValue); - } - public String[] getStringArray(String name) { return mProperties.getStringArray(name); } From 0126a60d010821e5247a1894d9b036250b2aee84 Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Thu, 3 Nov 2016 06:02:40 -0500 Subject: [PATCH 235/330] fixup! Make consumer group prefix on statsd metrics optional. --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 0559c2675..a5b282cba 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -342,7 +342,7 @@ public String getStatsDHostPort() { } public boolean getStatsDPrefixWithConsumerGroup(){ - return getBoolean("statsd.prefixWithConsumerGroup", true); + return getBoolean("statsd.prefixWithConsumerGroup"); } public String getMonitoringBlacklistTopics() { From 13e79e1084087e63fb35aefbda3f45663c0ef057 Mon Sep 17 00:00:00 2001 From: Dan Goldin Date: Thu, 3 Nov 2016 22:24:44 -0400 Subject: [PATCH 236/330] Support build profiles for Kafka 0.8 and 0.10. Write tests (but comment both out for now) --- README.md | 4 + pom.xml | 89 +++++- .../secor/performance/PerformanceTest.java | 277 ----------------- .../secor/performance/PerformanceTest010.java | 278 ++++++++++++++++++ .../secor/performance/PerformanceTest08.java | 273 +++++++++++++++++ 5 files changed, 632 insertions(+), 289 deletions(-) delete mode 100644 src/test/java/com/pinterest/secor/performance/PerformanceTest.java create mode 100644 src/test/java/com/pinterest/secor/performance/PerformanceTest010.java create mode 100644 src/test/java/com/pinterest/secor/performance/PerformanceTest08.java diff --git a/README.md b/README.md index f3e394503..b6451552c 100644 --- a/README.md +++ b/README.md @@ -29,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.8 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.10 client you should use the kafka-0.10 profile +mvn -Pkafka-0.10 package ``` ##### Run tests (optional) diff --git a/pom.xml b/pom.xml index 2694d4132..d6ee35854 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.24-SNAPSHOT + 0.23-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter @@ -106,17 +106,6 @@ jets3t 0.9.4 - - org.apache.kafka - kafka_2.10 - 0.10.0.1 - - - org.slf4j - slf4j-simple - - - log4j log4j @@ -532,6 +521,9 @@ release + + true + @@ -576,6 +568,79 @@ + + + org.apache.kafka + kafka_2.10 + 0.8.2.1 + + + org.slf4j + slf4j-simple + + + + + + + kafka-0.10 + + + + 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.0.1 + + + org.slf4j + slf4j-simple + + + + diff --git a/src/test/java/com/pinterest/secor/performance/PerformanceTest.java b/src/test/java/com/pinterest/secor/performance/PerformanceTest.java deleted file mode 100644 index 560c16f85..000000000 --- a/src/test/java/com/pinterest/secor/performance/PerformanceTest.java +++ /dev/null @@ -1,277 +0,0 @@ -/** - * 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.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 PerformanceTest { - - public static void main(String[] args) throws Exception { - if (args.length != 4) { - System.err.println("USAGE: java " + PerformanceTest.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/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 From 0e23190e110a94eac0576ce0a3261e4ede361b84 Mon Sep 17 00:00:00 2001 From: Dan Goldin Date: Thu, 3 Nov 2016 22:35:36 -0400 Subject: [PATCH 237/330] remove gpg from Kafka 0.10 profile --- pom.xml | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/pom.xml b/pom.xml index d6ee35854..13bae2972 100644 --- a/pom.xml +++ b/pom.xml @@ -612,20 +612,6 @@ - - org.apache.maven.plugins - maven-gpg-plugin - 1.5 - - - sign-artifacts - verify - - sign - - - - From 6cb0c6de8e75b8f0b8a0b5b0f7bd9dc18fc171f2 Mon Sep 17 00:00:00 2001 From: Liam Stewart Date: Thu, 3 Nov 2016 05:41:05 -0500 Subject: [PATCH 238/330] Allow auto.offset.reset to be specified in configuration. In most cases, the value should not be set to anything other than "smallest". However, "largest" can be useful when transitioning from to secor from another logsaver in situations where everything up to largest has already been saved. --- src/main/config/secor.common.properties | 8 ++++++++ src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../java/com/pinterest/secor/reader/MessageReader.java | 4 +--- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index e89a8e75f..aa8b4f3ef 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -128,6 +128,14 @@ 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. diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 9fcc5b2b0..9a473b8d1 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -91,6 +91,10 @@ 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"); } diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 95f0c9db7..073e81ac4 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -116,9 +116,7 @@ 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()); // Properties required to upgrade from kafka 0.8.x to 0.9.x From 68db8e16086b5964a42af2a66c24432685183618 Mon Sep 17 00:00:00 2001 From: jaimess Date: Tue, 8 Nov 2016 09:41:25 +0100 Subject: [PATCH 239/330] thrift version back to 0.5.0 parquet version back to 1.8.1 --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 3659fd304..c458fb1de 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ 1.6 UTF-8 UTF-8 - 1.9.0 + 1.8.1 @@ -240,7 +240,7 @@ org.apache.thrift libthrift - 0.9.3 + 0.5.0 com.twitter From 0e1338683107312c657156756805230118c03e9c Mon Sep 17 00:00:00 2001 From: Dan Goldin Date: Tue, 8 Nov 2016 21:35:33 -0500 Subject: [PATCH 240/330] Added a new 0.8 dev profile without gpg --- pom.xml | 52 +++++++++++++++++++++++++++++++++++++++++++++++++--- 1 file changed, 49 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 13bae2972..3dea05033 100644 --- a/pom.xml +++ b/pom.xml @@ -521,9 +521,6 @@ release - - true - @@ -582,6 +579,55 @@ + + kafka-0.8-dev + + true + + + + + 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.kafka + kafka_2.10 + 0.8.2.1 + + + org.slf4j + slf4j-simple + + + + + kafka-0.10 From 6a826b3e48414a988bc9daba94789a79998bc952 Mon Sep 17 00:00:00 2001 From: Dan Goldin Date: Wed, 9 Nov 2016 21:19:38 -0500 Subject: [PATCH 241/330] Updated dev profiles to remove unused plugins --- README.md | 4 ++-- pom.xml | 62 +------------------------------------------------------ 2 files changed, 3 insertions(+), 63 deletions(-) diff --git a/README.md b/README.md index b6451552c..1bde299dc 100644 --- a/README.md +++ b/README.md @@ -34,8 +34,8 @@ 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.10 client you should use the kafka-0.10 profile -mvn -Pkafka-0.10 package +# To use the Kafka 0.10 client you should use the kafka-0.10-dev profile +mvn -Pkafka-0.10-dev package ``` ##### Run tests (optional) diff --git a/pom.xml b/pom.xml index 3dea05033..bce6c00bc 100644 --- a/pom.xml +++ b/pom.xml @@ -584,36 +584,6 @@ true - - - - 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.kafka @@ -629,37 +599,7 @@ - kafka-0.10 - - - - 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 - - - - - - + kafka-0.10-dev org.apache.kafka From 33a08dd1c9a9114a42405f63def42d44257eaa79 Mon Sep 17 00:00:00 2001 From: Laust Rud Jacobsen Date: Mon, 21 Nov 2016 11:12:46 +0100 Subject: [PATCH 242/330] README: add newlines for human readability of cmd --- README.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 0bc213dbc..6c7dc4581 100644 --- a/README.md +++ b/README.md @@ -47,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 From 3ba27566956332fba4fd630eb8e2a06c4f59a95e Mon Sep 17 00:00:00 2001 From: Doron Levi Date: Sun, 27 Nov 2016 15:05:36 +0200 Subject: [PATCH 243/330] Change some members visibility for better extension --- .../pinterest/secor/consumer/Consumer.java | 22 +++++++-------- .../parser/TimestampedMessageParser.java | 28 +++++++++---------- .../pinterest/secor/reader/MessageReader.java | 16 +++++------ .../secor/uploader/HadoopS3UploadManager.java | 2 +- .../pinterest/secor/uploader/Uploader.java | 16 +++++------ .../pinterest/secor/writer/MessageWriter.java | 14 +++++----- 6 files changed, 49 insertions(+), 49 deletions(-) diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 7a3d40856..9bb83f951 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -50,16 +50,16 @@ public class Consumer extends Thread { private static final Logger LOG = LoggerFactory.getLogger(Consumer.class); - private SecorConfig mConfig; - - private MessageReader mMessageReader; - private MessageWriter mMessageWriter; - private MessageParser mMessageParser; - private OffsetTracker mOffsetTracker; - private MessageTransformer mMessageTransformer; - private Uploader mUploader; + protected SecorConfig mConfig; + + 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; @@ -109,7 +109,7 @@ public void run() { checkUploadPolicy(); } - private void checkUploadPolicy() { + protected void checkUploadPolicy() { try { mUploader.applyPolicy(); } catch (Exception e) { @@ -118,7 +118,7 @@ private void checkUploadPolicy() { } // @return whether there are more messages left to consume - private boolean consumeNextMessage() { + protected boolean consumeNextMessage() { Message rawMessage = null; try { boolean hasNext = mMessageReader.hasNext(); diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index cf992bcbe..722b381ca 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -40,20 +40,20 @@ public abstract class TimestampedMessageParser extends MessageParser implements * SimpleDateFormat are NOT thread-safe. * Each parser needs to have their own local SimpleDateFormat or it'll cause race condition. */ - private final SimpleDateFormat mDtFormatter; - private final SimpleDateFormat mHrFormatter; - private final SimpleDateFormat mDtHrFormatter; - private final int mFinalizerDelaySeconds; - private final SimpleDateFormat mDtHrMinFormatter; - private final SimpleDateFormat mMinFormatter; - - private final String mDtFormat; - private final String mHrFormat; - private final String mMinFormat; - - private final String mDtPrefix; - private final String mHrPrefix; - private final String mMinPrefix; + 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; diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 073e81ac4..4efcb6512 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -50,14 +50,14 @@ 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; - private final int mTopicPartitionForgetSeconds; - private final int mCheckMessagesPerSecond; - private int mNMessages; + 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; public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws UnknownHostException { diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java index 16771d1d2..166795d1a 100644 --- a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -37,7 +37,7 @@ public class HadoopS3UploadManager extends UploadManager { private static final Logger LOG = LoggerFactory.getLogger(HadoopS3UploadManager.class); - private static final ExecutorService executor = Executors.newFixedThreadPool(256); + protected static final ExecutorService executor = Executors.newFixedThreadPool(256); public HadoopS3UploadManager(SecorConfig config) { super(config); diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index 37dc7add7..d923d444c 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -42,12 +42,12 @@ public class Uploader { private static final Logger LOG = LoggerFactory.getLogger(Uploader.class); - private SecorConfig mConfig; - private OffsetTracker mOffsetTracker; - private FileRegistry mFileRegistry; - private ZookeeperConnector mZookeeperConnector; - private UploadManager mUploadManager; - private String mTopicFilter; + protected SecorConfig mConfig; + protected OffsetTracker mOffsetTracker; + protected FileRegistry mFileRegistry; + protected ZookeeperConnector mZookeeperConnector; + protected UploadManager mUploadManager; + protected String mTopicFilter; /** @@ -77,7 +77,7 @@ public void init(SecorConfig config, OffsetTracker offsetTracker, FileRegistry f } - private void uploadFiles(TopicPartition topicPartition) throws Exception { + protected void uploadFiles(TopicPartition topicPartition) throws Exception { long committedOffsetCount = mOffsetTracker.getTrueCommittedOffsetCount(topicPartition); long lastSeenOffset = mOffsetTracker.getLastSeenOffset(topicPartition); @@ -204,7 +204,7 @@ private boolean isRequiredToUploadAtTime(TopicPartition topicPartition) throws E return false; } - private void checkTopicPartition(TopicPartition topicPartition) throws Exception { + 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); diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index 31a3fe50a..d29321fcf 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -42,13 +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; - private String mLocalPrefix; - private final int mGeneration; + 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 { From e37e5ff489d0a895ea7a4f4fadd7771229c1d99f Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Mon, 28 Nov 2016 15:43:44 +0800 Subject: [PATCH 244/330] flexible delimiter added --- src/main/config/secor.common.properties | 3 +- .../pinterest/secor/common/SecorConfig.java | 8 +- ...xibleDelimitedFileReaderWriterFactory.java | 136 ++++++++++++++++++ 3 files changed, 144 insertions(+), 3 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index e287cedb7..b5ceeb96c 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -270,7 +270,7 @@ statsd.hostport= # TBinaryProtocol by default secor.thrift.protocol.class= -# Thrift message class. It applies to parquet output. +# 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.*= @@ -327,6 +327,7 @@ 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 +secor.file.reader.writer.Delimiter='' # 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index a9e37f705..05a4fcb6d 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -418,6 +418,10 @@ public String getFileReaderWriterFactory() { return getString("secor.file.reader.writer.factory"); } + public String getFileReaderWriterDelimiter(){ + return getString("secor.file.reader.writer.Delimiter"); + } + public String getPerfTestTopicPrefix() { return getString("secor.kafka.perf_topic_prefix"); } @@ -490,8 +494,8 @@ public Map getThriftMessageClassPerTopic() { 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); 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..88bc2c4e8 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -0,0 +1,136 @@ +/** + * 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 FlexibleDelimitedFileReaderWriterFactory implements FileReaderWriterFactory { + private static byte DELIMITER = SecorConfig.load().getFileReaderWriterDelimiter(); + + @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; + + 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(); + } + + @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 FlexibleDelimitedFileWriter implements FileWriter { + private final CountingOutputStream mCountingStream; + private final BufferedOutputStream mWriter; + private Compressor mCompressor = null; + + 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))); + } + + @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()); + this.mWriter.write(DELIMITER); + } + + @Override + public void close() throws IOException { + this.mWriter.close(); + CodecPool.returnCompressor(mCompressor); + mCompressor = null; + } + } +} From 8d4d59723a612a69ef44ed9cffa11917dcd958b0 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Mon, 28 Nov 2016 16:48:41 +0800 Subject: [PATCH 245/330] added flexible delimiter support --- src/main/config/secor.common.properties | 3 +- .../pinterest/secor/common/SecorConfig.java | 8 ++++-- ...xibleDelimitedFileReaderWriterFactory.java | 28 +++++++++++++++++-- .../parser/DailyOffsetMessageParser.java | 2 +- .../parser/PartitionedMessageParser.java | 2 +- 5 files changed, 35 insertions(+), 8 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index b5ceeb96c..c6ed4a779 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -327,7 +327,8 @@ 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 -secor.file.reader.writer.Delimiter='' +secor.file.reader.Delimiter=\n +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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 05a4fcb6d..76519ec78 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -418,8 +418,12 @@ public String getFileReaderWriterFactory() { return getString("secor.file.reader.writer.factory"); } - public String getFileReaderWriterDelimiter(){ - return getString("secor.file.reader.writer.Delimiter"); + public String getFileReaderDelimiter(){ + return getString("secor.file.reader.Delimiter"); + } + + public String getFileWriterDelimiter(){ + return getString("secor.file.writer.Delimiter"); } public String getPerfTestTopicPrefix() { diff --git a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java index 88bc2c4e8..c2a69f46a 100644 --- a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -33,18 +33,20 @@ 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; /** - * Delimited Text File Reader Writer with Compression + * Flexible Delimited Text File Reader Writer with Compression * - * @author Praveen Murugesan (praveen@uber.com) + * @author Ahsan Nabi Dar (ahsan@wego.com) */ public class FlexibleDelimitedFileReaderWriterFactory implements FileReaderWriterFactory { - private static byte DELIMITER = SecorConfig.load().getFileReaderWriterDelimiter(); @Override public FileReader BuildFileReader(LogFilePath logFilePath, CompressionCodec codec) @@ -57,10 +59,13 @@ public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec code return new FlexibleDelimitedFileWriter(logFilePath, codec); } + + protected class FlexibleDelimitedFileReader implements FileReader { private final BufferedInputStream mReader; private long mOffset; private Decompressor mDecompressor = null; + private byte DELIMITER = delimterReader(); public FlexibleDelimitedFileReader(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -73,6 +78,14 @@ public FlexibleDelimitedFileReader(LogFilePath path, CompressionCodec codec) thr this.mOffset = path.getOffset(); } + public byte delimterReader() { + byte delimiter = '\n'; + try { + delimiter = Byte.valueOf(SecorConfig.load().getFileReaderDelimiter()); + }catch(ConfigurationException e){} + return delimiter; + } + @Override public KeyValue next() throws IOException { ByteArrayOutputStream messageBuffer = new ByteArrayOutputStream(); @@ -103,6 +116,7 @@ protected class FlexibleDelimitedFileWriter implements FileWriter { private final CountingOutputStream mCountingStream; private final BufferedOutputStream mWriter; private Compressor mCompressor = null; + private byte DELIMITER = delimterWriter(); public FlexibleDelimitedFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -114,6 +128,14 @@ public FlexibleDelimitedFileWriter(LogFilePath path, CompressionCodec codec) thr mCompressor = CodecPool.getCompressor(codec))); } + public byte delimterWriter() { + byte delimiter = '\n'; + try { + delimiter = Byte.valueOf(SecorConfig.load().getFileWriterDelimiter()); + }catch(ConfigurationException e){} + return delimiter; + } + @Override public long getLength() throws IOException { assert this.mCountingStream != null; diff --git a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java index 94d9e8a09..e653c6695 100644 --- a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java @@ -28,7 +28,7 @@ /** * Offset message parser groups messages based on the offset ranges. * - * @author Pawel Garbacki (pawel@pinterest.com) + * @author Ahsan Nabi Dar (ahsan@wego.com) */ public class DailyOffsetMessageParser extends TimestampedMessageParser { diff --git a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java index c193faa28..508db3f1c 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java @@ -28,7 +28,7 @@ /** * Offset message parser groups messages based on the offset ranges. * - * @author Pawel Garbacki (pawel@pinterest.com) + * @author Ahsan Nabi Dar (ahsan@wego.com) */ public class PartitionedMessageParser extends TimestampedMessageParser { From ce36f7c2ff4563c058a293defbc3f7610fdb0c43 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Tue, 29 Nov 2016 11:13:10 +0800 Subject: [PATCH 246/330] added null check --- ...xibleDelimitedFileReaderWriterFactory.java | 215 +++++++++--------- 1 file changed, 110 insertions(+), 105 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java index c2a69f46a..cfab1a355 100644 --- a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -1,19 +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. - */ +* 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; @@ -42,117 +42,122 @@ import com.pinterest.secor.common.SecorConfig; /** - * Flexible Delimited Text File Reader Writer with Compression - * - * @author Ahsan Nabi Dar (ahsan@wego.com) - */ +* 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 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 DELIMITER = delimterReader(); + + 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(); } - @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 DELIMITER = delimterReader(); - - 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 delimterReader() { - byte delimiter = '\n'; - try { - delimiter = Byte.valueOf(SecorConfig.load().getFileReaderDelimiter()); - }catch(ConfigurationException e){} - return delimiter; - } - - @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); + public byte delimterReader() { + byte delimiter = '\n'; + try { + delimiter = Byte.valueOf(SecorConfig.load().getFileReaderDelimiter()); + }catch(ConfigurationException e){} + return delimiter; + } + + @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"); } - return new KeyValue(this.mOffset++, messageBuffer.toByteArray()); - } - - @Override - public void close() throws IOException { - this.mReader.close(); - CodecPool.returnDecompressor(mDecompressor); - mDecompressor = null; + } + 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 DELIMITER = delimterWriter(); - - 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 delimterWriter() { - byte delimiter = '\n'; - try { - delimiter = Byte.valueOf(SecorConfig.load().getFileWriterDelimiter()); - }catch(ConfigurationException e){} - return delimiter; + private final CountingOutputStream mCountingStream; + private final BufferedOutputStream mWriter; + private Compressor mCompressor = null; + private byte DELIMITER = delimterWriter(); + + 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 delimterWriter() { + byte delimiter = '\\'; + try { + String writerDelimiter = SecorConfig.load().getFileWriterDelimiter(); + if (!writerDelimiter.isEmpty()){ + delimiter = Byte.valueOf(writerDelimiter); + } + }catch(ConfigurationException e){} + return delimiter; } @Override public long getLength() throws IOException { - assert this.mCountingStream != null; - return this.mCountingStream.getCount(); + assert this.mCountingStream != null; + return this.mCountingStream.getCount(); } @Override public void write(KeyValue keyValue) throws IOException { - this.mWriter.write(keyValue.getValue()); + this.mWriter.write(keyValue.getValue()); + if (DELIMITER != 92){ this.mWriter.write(DELIMITER); + } } @Override public void close() throws IOException { - this.mWriter.close(); - CodecPool.returnCompressor(mCompressor); - mCompressor = null; + this.mWriter.close(); + CodecPool.returnCompressor(mCompressor); + mCompressor = null; } + } } -} From b00d31755dda0f3982f711bda8357d6bfd9f5097 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Tue, 29 Nov 2016 11:28:46 +0800 Subject: [PATCH 247/330] updated byte conversion logic --- .../io/impl/FlexibleDelimitedFileReaderWriterFactory.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java index cfab1a355..3c8520b89 100644 --- a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -81,7 +81,10 @@ public FlexibleDelimitedFileReader(LogFilePath path, CompressionCodec codec) thr public byte delimterReader() { byte delimiter = '\n'; try { - delimiter = Byte.valueOf(SecorConfig.load().getFileReaderDelimiter()); + String readerDelimiter = SecorConfig.load().getFileReaderDelimiter(); + if (!readerDelimiter.isEmpty()){ + delimiter = (byte)readerDelimiter.charAt(0); + } }catch(ConfigurationException e){} return delimiter; } @@ -133,7 +136,7 @@ public byte delimterWriter() { try { String writerDelimiter = SecorConfig.load().getFileWriterDelimiter(); if (!writerDelimiter.isEmpty()){ - delimiter = Byte.valueOf(writerDelimiter); + delimiter = (byte)writerDelimiter.charAt(0); } }catch(ConfigurationException e){} return delimiter; From 18aa59f2e9ab040640f0655d7f28182dcc7d6e4f Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Wed, 30 Nov 2016 10:01:47 +0800 Subject: [PATCH 248/330] updated byte match logic for value and updated config --- src/main/config/secor.common.properties | 3 +++ .../io/impl/FlexibleDelimitedFileReaderWriterFactory.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index c6ed4a779..a51d64cbd 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -327,7 +327,10 @@ 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. diff --git a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java index 3c8520b89..44e4433f4 100644 --- a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -151,7 +151,7 @@ public long getLength() throws IOException { @Override public void write(KeyValue keyValue) throws IOException { this.mWriter.write(keyValue.getValue()); - if (DELIMITER != 92){ + if (DELIMITER != (byte)'\\'){ this.mWriter.write(DELIMITER); } } From 994fa7177364c704347ad2a193336c056dc81bdd Mon Sep 17 00:00:00 2001 From: kpavel Date: Wed, 30 Nov 2016 17:02:58 +0200 Subject: [PATCH 249/330] Fix DelimitedTextFileWriter.getLength Currently, at least for DelimitedTextFileWriter, the getLength returned value doesn't take into account data that not flushed yet. As a result, in Uploader.checkTopicPartition wrong decision can happen here: "size >= mConfig.getMaxFileSizeBytes()" when data already cashed and much larger than "secor.max.file.size.bytes". --- .../secor/io/impl/DelimitedTextFileReaderWriterFactory.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java index 42e7e9d20..b4c23a032 100644 --- a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java @@ -117,6 +117,7 @@ public DelimitedTextFileWriter(LogFilePath path, CompressionCodec codec) throws @Override public long getLength() throws IOException { assert this.mCountingStream != null; + this.mWriter.flush(); return this.mCountingStream.getCount(); } From cafc6b1b602f2611d3b568bea3895d4b8878c038 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Fri, 2 Dec 2016 10:06:27 +0800 Subject: [PATCH 250/330] updated based on review --- .../pinterest/secor/common/SecorConfig.java | 12 +- ...xibleDelimitedFileReaderWriterFactory.java | 136 +++++++++--------- 2 files changed, 80 insertions(+), 68 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 76519ec78..5137a50e6 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -419,11 +419,19 @@ public String getFileReaderWriterFactory() { } public String getFileReaderDelimiter(){ - return getString("secor.file.reader.Delimiter"); + 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(){ - return getString("secor.file.writer.Delimiter"); + 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() { diff --git a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java index 44e4433f4..7a25efbda 100644 --- a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -65,7 +65,7 @@ protected class FlexibleDelimitedFileReader implements FileReader { private final BufferedInputStream mReader; private long mOffset; private Decompressor mDecompressor = null; - private byte DELIMITER = delimterReader(); + private byte DELIMITER = getReaderDelimiter(); public FlexibleDelimitedFileReader(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -78,89 +78,93 @@ public FlexibleDelimitedFileReader(LogFilePath path, CompressionCodec codec) thr this.mOffset = path.getOffset(); } - public byte delimterReader() { + public byte getReaderDelimiter() { byte delimiter = '\n'; try { String readerDelimiter = SecorConfig.load().getFileReaderDelimiter(); if (!readerDelimiter.isEmpty()){ delimiter = (byte)readerDelimiter.charAt(0); } - }catch(ConfigurationException e){} - return delimiter; + } 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()) != 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"); - } + @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()); + messageBuffer.write(nextByte); } + return new KeyValue(this.mOffset++, messageBuffer.toByteArray()); + } - @Override - public void close() throws IOException { - this.mReader.close(); - CodecPool.returnDecompressor(mDecompressor); - mDecompressor = null; - } + @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 DELIMITER = delimterWriter(); - - 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))); - } + protected class FlexibleDelimitedFileWriter implements FileWriter { + private final CountingOutputStream mCountingStream; + private final BufferedOutputStream mWriter; + private Compressor mCompressor = null; + private byte DELIMITER = getWriterDelimiter(); - public byte delimterWriter() { - byte delimiter = '\\'; - try { - String writerDelimiter = SecorConfig.load().getFileWriterDelimiter(); - if (!writerDelimiter.isEmpty()){ - delimiter = (byte)writerDelimiter.charAt(0); - } - }catch(ConfigurationException e){} - return delimiter; - } + 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))); + } - @Override - public long getLength() throws IOException { - assert this.mCountingStream != null; - return this.mCountingStream.getCount(); + public byte getWriterDelimiter() { + byte delimiter = '\\'; + try { + String writerDelimiter = SecorConfig.load().getFileWriterDelimiter(); + if (!writerDelimiter.isEmpty()){ + delimiter = (byte)writerDelimiter.charAt(0); } + } catch(ConfigurationException e) { + throw new RuntimeException("Error loading configuration from getFileWriterDelimiter()"); + } + return delimiter; + } - @Override - public void write(KeyValue keyValue) throws IOException { - this.mWriter.write(keyValue.getValue()); - if (DELIMITER != (byte)'\\'){ - this.mWriter.write(DELIMITER); - } - } + @Override + public long getLength() throws IOException { + assert this.mCountingStream != null; + return this.mCountingStream.getCount(); + } - @Override - public void close() throws IOException { - this.mWriter.close(); - CodecPool.returnCompressor(mCompressor); - mCompressor = null; - } + @Override + public void write(KeyValue keyValue) throws IOException { + this.mWriter.write(keyValue.getValue()); + if (DELIMITER != (byte)'\\'){ + this.mWriter.write(DELIMITER); } } + + @Override + public void close() throws IOException { + this.mWriter.close(); + CodecPool.returnCompressor(mCompressor); + mCompressor = null; + } + } +} From 03e5e5b300e44a168f1d3b83dd35ecdcac591412 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Mon, 5 Dec 2016 10:16:37 +0800 Subject: [PATCH 251/330] updated delimiter write logic with a use of boolean --- .../impl/FlexibleDelimitedFileReaderWriterFactory.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java index 7a25efbda..53dcc32a3 100644 --- a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -122,6 +122,7 @@ protected class FlexibleDelimitedFileWriter implements FileWriter { private final BufferedOutputStream mWriter; private Compressor mCompressor = null; private byte DELIMITER = getWriterDelimiter(); + private boolean addDelimiter = true; public FlexibleDelimitedFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -134,10 +135,12 @@ public FlexibleDelimitedFileWriter(LogFilePath path, CompressionCodec codec) thr } public byte getWriterDelimiter() { - byte delimiter = '\\'; + byte delimiter = '\n'; try { String writerDelimiter = SecorConfig.load().getFileWriterDelimiter(); - if (!writerDelimiter.isEmpty()){ + if (writerDelimiter.isEmpty()){ + addDelimiter = false; + } else { delimiter = (byte)writerDelimiter.charAt(0); } } catch(ConfigurationException e) { @@ -155,7 +158,7 @@ public long getLength() throws IOException { @Override public void write(KeyValue keyValue) throws IOException { this.mWriter.write(keyValue.getValue()); - if (DELIMITER != (byte)'\\'){ + if (addDelimiter){ this.mWriter.write(DELIMITER); } } From 05dcc68fa53b7bb5222fb12d48262ddb231b36e7 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Tue, 6 Dec 2016 10:18:57 +0800 Subject: [PATCH 252/330] updated for lexible delimiter --- ...lexibleDelimitedFileReaderWriterFactory.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java index 53dcc32a3..72ffe49c3 100644 --- a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -65,7 +65,7 @@ protected class FlexibleDelimitedFileReader implements FileReader { private final BufferedInputStream mReader; private long mOffset; private Decompressor mDecompressor = null; - private byte DELIMITER = getReaderDelimiter(); + private byte mDelimiter = getReaderDelimiter(); public FlexibleDelimitedFileReader(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -95,7 +95,7 @@ public byte getReaderDelimiter() { public KeyValue next() throws IOException { ByteArrayOutputStream messageBuffer = new ByteArrayOutputStream(); int nextByte; - while ((nextByte = mReader.read()) != DELIMITER) { + while ((nextByte = mReader.read()) != mDelimiter) { if (nextByte == -1) { // end of stream? if (messageBuffer.size() == 0) { // if no byte read return null; @@ -121,8 +121,8 @@ protected class FlexibleDelimitedFileWriter implements FileWriter { private final CountingOutputStream mCountingStream; private final BufferedOutputStream mWriter; private Compressor mCompressor = null; - private byte DELIMITER = getWriterDelimiter(); - private boolean addDelimiter = true; + private byte mDelimiter = getWriterDelimiter(); + private boolean addDelimiter = false; public FlexibleDelimitedFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { Path fsPath = new Path(path.getLogFilePath()); @@ -135,12 +135,11 @@ public FlexibleDelimitedFileWriter(LogFilePath path, CompressionCodec codec) thr } public byte getWriterDelimiter() { - byte delimiter = '\n'; + byte delimiter = '\\'; try { String writerDelimiter = SecorConfig.load().getFileWriterDelimiter(); - if (writerDelimiter.isEmpty()){ - addDelimiter = false; - } else { + if (!writerDelimiter.isEmpty()){ + addDelimiter = true; delimiter = (byte)writerDelimiter.charAt(0); } } catch(ConfigurationException e) { @@ -159,7 +158,7 @@ public long getLength() throws IOException { public void write(KeyValue keyValue) throws IOException { this.mWriter.write(keyValue.getValue()); if (addDelimiter){ - this.mWriter.write(DELIMITER); + this.mWriter.write(mDelimiter); } } From deb583f658db503909b9840633965b4459ed02fc Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Wed, 7 Dec 2016 13:35:10 +0800 Subject: [PATCH 253/330] changed \ to \n --- .../secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java index 72ffe49c3..d85f99233 100644 --- a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -135,7 +135,7 @@ public FlexibleDelimitedFileWriter(LogFilePath path, CompressionCodec codec) thr } public byte getWriterDelimiter() { - byte delimiter = '\\'; + byte delimiter = '\n'; try { String writerDelimiter = SecorConfig.load().getFileWriterDelimiter(); if (!writerDelimiter.isEmpty()){ From dd5d74420f2b847bf5045968885967b953834eb3 Mon Sep 17 00:00:00 2001 From: rob mccoll Date: Fri, 20 Jan 2017 16:21:32 -0500 Subject: [PATCH 254/330] adds path style access --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ .../java/com/pinterest/secor/uploader/S3UploadManager.java | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 5137a50e6..c4ceff31b 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -273,6 +273,10 @@ public String getAwsRole() { return getString("aws.role"); } + public boolean getAwsClientPathStyleAccess() { + return getBoolean("aws.client.pathstyleaccess", false); + } + public boolean getAwsProxyEnabled(){ return getBoolean("aws.proxy.isEnabled"); } diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java index 7d8aa7814..4d43d9f9d 100644 --- a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -28,6 +28,7 @@ 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; @@ -118,6 +119,12 @@ public void refresh() {} 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()) { From 252f7194431c334b25066601ee7a42dd10664a3f Mon Sep 17 00:00:00 2001 From: rob mccoll Date: Fri, 20 Jan 2017 16:31:54 -0500 Subject: [PATCH 255/330] AWS S3 path style access config + comment --- src/main/config/secor.common.properties | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index a51d64cbd..4778d40ac 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -54,6 +54,10 @@ aws.proxy.http.port= 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 # ########################### From a88dd58d89e387c9e487b02d688217956c217fc2 Mon Sep 17 00:00:00 2001 From: rob mccoll Date: Mon, 23 Jan 2017 16:21:05 -0500 Subject: [PATCH 256/330] if no prefix/topic, don't add empty component to path --- src/main/java/com/pinterest/secor/common/LogFilePath.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java index 3d9aa8f9c..21911b74d 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -146,8 +146,12 @@ public LogFilePath withPrefix(String prefix) { public String getLogFileParentDir() { ArrayList elements = new ArrayList(); - elements.add(mPrefix); - elements.add(mTopic); + if(mPrefix.length() > 0) { + elements.add(mPrefix); + } + if(mTopic.length() > 0) { + elements.add(mTopic); + } return StringUtils.join(elements, "/"); } From 88e522ac6166abe0f807d5fb2497fa13c0847a0e Mon Sep 17 00:00:00 2001 From: Rob McColl Date: Mon, 23 Jan 2017 18:23:44 -0500 Subject: [PATCH 257/330] style, null check for extra safety --- src/main/java/com/pinterest/secor/common/LogFilePath.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java index 21911b74d..6f0e8b14c 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -146,10 +146,10 @@ public LogFilePath withPrefix(String prefix) { public String getLogFileParentDir() { ArrayList elements = new ArrayList(); - if(mPrefix.length() > 0) { + if (mPrefix != null && mPrefix.length() > 0) { elements.add(mPrefix); } - if(mTopic.length() > 0) { + if (mTopic != null && mTopic.length() > 0) { elements.add(mTopic); } return StringUtils.join(elements, "/"); From 06099b4ceecea1a84c422ed51d7df5d791adabd7 Mon Sep 17 00:00:00 2001 From: Josh Becker Date: Tue, 24 Jan 2017 13:55:19 -0500 Subject: [PATCH 258/330] zookerper should be zookeeper --- src/main/config/secor.prod.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/config/secor.prod.properties b/src/main/config/secor.prod.properties index f2806b2b9..8ce4718f8 100644 --- a/src/main/config/secor.prod.properties +++ b/src/main/config/secor.prod.properties @@ -23,7 +23,7 @@ include=secor.common.properties # TODO(pawel): use a list of nodes or even better, extract active brokers from zookeeper. kafka.seed.broker.host= -# List of Kafka Zookeeper servers with ports :. +# List of Kafka Zookeeper servers with ports :. zookeeper.quorum= # Fill the section which fits your needs From 66e24e80e2f3e6a2b5302e891dd637fab5436e3f Mon Sep 17 00:00:00 2001 From: Duy Nguyen Date: Tue, 7 Feb 2017 14:44:34 -0500 Subject: [PATCH 259/330] Added configurable date prefixes I also added `testDatePrefix()`, fixed existing tests that were failing, and made minor formatting changes (e.g. indentation) to conform with the rest of the package. --- .../secor/parser/DateMessageParser.java | 18 +++++--- .../secor/parser/DateMessageParserTest.java | 44 +++++++++++-------- 2 files changed, 37 insertions(+), 25 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java index 74067994f..8afbf2030 100644 --- a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java @@ -30,7 +30,7 @@ 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 @@ -39,20 +39,26 @@ * */ 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 = new SimpleDateFormat(defaultFormatter); 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.setTimeZone(timeZone); + + mDtPrefix = (TimestampedMessageParser.usingDatePrefix(config) == null) ? "" : TimestampedMessageParser.usingDatePrefix(config); } @Override @@ -65,12 +71,10 @@ public String[] extractPartitions(Message message) { if (fieldValue != null && inputPattern != null) { try { 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 = {} for the input pattern = {} . Using date default = {}", - fieldValue.toString(), inputPattern.toString(), result[0]); + LOG.warn("Impossible to convert date = {} with the input pattern = {}. Using date default = {}", + fieldValue.toString(), inputPattern.toString(), result[0]); } } } diff --git a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index 7db0ed92a..bae5ca375 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -45,45 +45,44 @@ public class DateMessageParserTest extends TestCase { public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); - + 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"); + .getBytes("UTF-8"); mFormat1 = new Message("test", 0, 0, null, format1); 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"); + .getBytes("UTF-8"); mFormat2 = new Message("test", 0, 0, null, format2); 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"); + .getBytes("UTF-8"); mFormat3 = new Message("test", 0, 0, null, format3); 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"); + .getBytes("UTF-8"); mInvalidDate = new Message("test", 0, 0, null, invalidDate); - + byte isoFormat[] = "{\"timestamp\":\"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"); + .getBytes("UTF-8"); mISOFormat = new Message("test", 0, 0, null, isoFormat); - + 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"); + .getBytes("UTF-8"); mNestedISOFormat = new Message("test", 0, 0, null, nestedISOFormat); - } @Test public void testExtractDateUsingInputPattern() throws Exception { Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); - + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd HH:mm:ss"); - assertEquals("dt=2014-07-30", new DateMessageParser(mConfig).extractPartitions(mFormat1)[0]); + assertEquals("2014-07-30", new DateMessageParser(mConfig).extractPartitions(mFormat1)[0]); Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy/MM/d"); - assertEquals("dt=2014-10-25", new DateMessageParser(mConfig).extractPartitions(mFormat2)[0]); + assertEquals("2014-10-25", new DateMessageParser(mConfig).extractPartitions(mFormat2)[0]); Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyyy.MMMMM.dd GGG hh:mm aaa"); - assertEquals("dt=2001-07-04", new DateMessageParser(mConfig).extractPartitions(mFormat3)[0]); + assertEquals("2001-07-04", new DateMessageParser(mConfig).extractPartitions(mFormat3)[0]); } @Test @@ -92,20 +91,29 @@ public void testExtractDateWithWrongEntries() throws Exception { // invalid date 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 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"); + + 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'"); - assertEquals("dt=2016-01-11", new DateMessageParser(mConfig).extractPartitions(mNestedISOFormat)[0]); + assertEquals("2016-01-11", new DateMessageParser(mConfig).extractPartitions(mNestedISOFormat)[0]); } } From a75117eaf21c6e85a9da005d1ebd41214f2c2afc Mon Sep 17 00:00:00 2001 From: Duy Nguyen Date: Tue, 7 Feb 2017 19:02:52 -0500 Subject: [PATCH 260/330] Fixed false positive test failures This was introduced by commit 66e24e80e2f3e6a2b5302e891dd637fab5436e3f. Calls to `getString()` were not mocked and therefore always returned `null`. --- .../com/pinterest/secor/parser/DateMessageParser.java | 2 +- .../pinterest/secor/parser/DateMessageParserTest.java | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java index 8afbf2030..8b3c9c022 100644 --- a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java @@ -58,7 +58,7 @@ public DateMessageParser(SecorConfig config) { inputFormatter.setTimeZone(timeZone); outputFormatter.setTimeZone(timeZone); - mDtPrefix = (TimestampedMessageParser.usingDatePrefix(config) == null) ? "" : TimestampedMessageParser.usingDatePrefix(config); + mDtPrefix = TimestampedMessageParser.usingDatePrefix(config); } @Override diff --git a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index bae5ca375..bff6263f1 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -74,15 +74,16 @@ public void setUp() throws Exception { @Test public void testExtractDateUsingInputPattern() throws Exception { Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getString("partitioner.granularity.date.prefix", "dt=")).thenReturn("dt="); Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd HH:mm:ss"); - assertEquals("2014-07-30", new DateMessageParser(mConfig).extractPartitions(mFormat1)[0]); + assertEquals("dt=2014-07-30", new DateMessageParser(mConfig).extractPartitions(mFormat1)[0]); Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy/MM/d"); - assertEquals("2014-10-25", new DateMessageParser(mConfig).extractPartitions(mFormat2)[0]); + assertEquals("dt=2014-10-25", new DateMessageParser(mConfig).extractPartitions(mFormat2)[0]); Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyyy.MMMMM.dd GGG hh:mm aaa"); - assertEquals("2001-07-04", new DateMessageParser(mConfig).extractPartitions(mFormat3)[0]); + assertEquals("dt=2001-07-04", new DateMessageParser(mConfig).extractPartitions(mFormat3)[0]); } @Test @@ -113,7 +114,8 @@ 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="); - assertEquals("2016-01-11", new DateMessageParser(mConfig).extractPartitions(mNestedISOFormat)[0]); + assertEquals("dt=2016-01-11", new DateMessageParser(mConfig).extractPartitions(mNestedISOFormat)[0]); } } From fda2d2bfaf38351d0607447d9a4cf250baf59ce8 Mon Sep 17 00:00:00 2001 From: Gwilym Evans Date: Sat, 11 Feb 2017 23:22:35 +0000 Subject: [PATCH 261/330] add nanoseconds test case iso8601 parser related to but does not fix https://github.com/pinterest/secor/issues/305 --- .../com/pinterest/secor/parser/Iso8601ParserTest.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java index 83eb57437..822cc8209 100644 --- a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java @@ -40,6 +40,7 @@ public class Iso8601ParserTest extends TestCase { private Message mInvalidDate; private Message mISOFormat; private Message mNestedISOFormat; + private Message mNanosecondISOFormat; private Message mMissingDate; private OngoingStubbing getTimestamp; @@ -47,7 +48,7 @@ public class Iso8601ParserTest extends TestCase { public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); - + 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); @@ -68,6 +69,10 @@ public void setUp() throws Exception { .getBytes("UTF-8"); mNestedISOFormat = new Message("test", 0, 0, null, nestedISOFormat); + byte nanosecondISOFormat[] = "{\"timestamp\":\"2006-01-02T23:59:59.999999999Z\"}" + .getBytes("UTF-8"); + mNanosecondISOFormat = new Message("test", 0, 0, null, nanosecondISOFormat); + 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); @@ -84,6 +89,7 @@ public void testExtractDate() throws Exception { assertEquals("dt=2014-07-29", new Iso8601MessageParser(mConfig).extractPartitions(mFormat2)[0]); assertEquals("dt=2001-07-04", new Iso8601MessageParser(mConfig).extractPartitions(mFormat3)[0]); assertEquals("dt=2016-03-02", new Iso8601MessageParser(mConfig).extractPartitions(mFormat4)[0]); + assertEquals("dt=2006-01-02", new Iso8601MessageParser(mConfig).extractPartitions(mNanosecondISOFormat)[0]); assertEquals("dt=1970-01-01", new Iso8601MessageParser(mConfig).extractPartitions(mInvalidDate)[0]); assertEquals("dt=1970-01-01", new Iso8601MessageParser(mConfig).extractPartitions(mMissingDate)[0]); } From f147b6b3eb9c9bd194571ea51b9e7de67bb78e73 Mon Sep 17 00:00:00 2001 From: Gwilym Evans Date: Tue, 14 Feb 2017 13:31:43 +0000 Subject: [PATCH 262/330] remove Iso8601ParserTest::mISOFormat it's not used anywhere --- src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java index 822cc8209..e3e412e62 100644 --- a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java @@ -38,7 +38,6 @@ public class Iso8601ParserTest extends TestCase { private Message mFormat3; private Message mFormat4; private Message mInvalidDate; - private Message mISOFormat; private Message mNestedISOFormat; private Message mNanosecondISOFormat; private Message mMissingDate; From 6997b63d5a7f41e627227b4725cd0d70983fcb56 Mon Sep 17 00:00:00 2001 From: Gwilym Evans Date: Tue, 14 Feb 2017 13:36:40 +0000 Subject: [PATCH 263/330] actually utilise DateMessageParserTest::mISOFormat it wasn't referenced anywhere, but makes sense as a test case --- .../com/pinterest/secor/parser/DateMessageParserTest.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index bff6263f1..9dcfc71bf 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -62,7 +62,7 @@ public void setUp() throws Exception { .getBytes("UTF-8"); mInvalidDate = new Message("test", 0, 0, null, invalidDate); - byte isoFormat[] = "{\"timestamp\":\"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\"}" + 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); @@ -84,6 +84,9 @@ public void testExtractDateUsingInputPattern() throws Exception { 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]); } @Test @@ -117,5 +120,5 @@ public void testNestedField() throws Exception { Mockito.when(mConfig.getString("partitioner.granularity.date.prefix", "dt=")).thenReturn("dt="); assertEquals("dt=2016-01-11", new DateMessageParser(mConfig).extractPartitions(mNestedISOFormat)[0]); - } + } } From 4e295e0e9821a7c7632dac0a917fc8a61c585d1c Mon Sep 17 00:00:00 2001 From: Gwilym Evans Date: Tue, 14 Feb 2017 23:25:05 +0000 Subject: [PATCH 264/330] add nanoseconds and optional suffix tests to check and cover existing behaviour --- .../pinterest/secor/parser/DateMessageParserTest.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index 9dcfc71bf..57e052838 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -38,6 +38,7 @@ public class DateMessageParserTest extends TestCase { private Message mFormat3; private Message mInvalidDate; private Message mISOFormat; + private Message mNanosecondISOFormat; private Message mNestedISOFormat; private OngoingStubbing getTimestamp; @@ -66,6 +67,10 @@ public void setUp() throws Exception { .getBytes("UTF-8"); mISOFormat = new Message("test", 0, 0, null, isoFormat); + byte nanosecondISOFormat[] = "{\"timestamp\":\"2006-01-02T23:59:59.999999999Z\"}" + .getBytes("UTF-8"); + mNanosecondISOFormat = new Message("test", 0, 0, null, nanosecondISOFormat); + 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); @@ -87,6 +92,12 @@ public void testExtractDateUsingInputPattern() throws Exception { 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 From b8f0dd7ece4b111216d4d1059ee7f3c4245b5aa5 Mon Sep 17 00:00:00 2001 From: Gwilym Evans Date: Wed, 15 Feb 2017 01:12:38 +0000 Subject: [PATCH 265/330] let DateMessageParser use partitioner.granularity.date.format the default 'yyyy-MM-dd' should match existing behaviour --- .../pinterest/secor/parser/DateMessageParser.java | 11 ++++++----- .../secor/parser/DateMessageParserTest.java | 15 +++++++++++++++ 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java index 8b3c9c022..e47f47406 100644 --- a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java @@ -32,18 +32,17 @@ /** * 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 = new SimpleDateFormat(defaultFormatter); + protected SimpleDateFormat outputFormatter; protected Object inputPattern; protected SimpleDateFormat inputFormatter; @@ -56,6 +55,8 @@ public DateMessageParser(SecorConfig config) { inputPattern = mConfig.getMessageTimestampInputPattern(); inputFormatter = new SimpleDateFormat(inputPattern.toString()); inputFormatter.setTimeZone(timeZone); + + outputFormatter = new SimpleDateFormat(TimestampedMessageParser.usingDateFormat(config)); outputFormatter.setTimeZone(timeZone); mDtPrefix = TimestampedMessageParser.usingDatePrefix(config); diff --git a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index 57e052838..373489bed 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -80,6 +80,7 @@ public void setUp() throws Exception { public void testExtractDateUsingInputPattern() throws Exception { 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]); @@ -103,6 +104,8 @@ public void testExtractDateUsingInputPattern() throws Exception { @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 Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd HH:mm:ss"); // any pattern assertEquals(DateMessageParser.defaultDate, new DateMessageParser( @@ -119,6 +122,7 @@ 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]); } @@ -129,7 +133,18 @@ public void testNestedField() throws Exception { 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]); + } } From 3221e886c706a77af744a5b9f458d0c6371f9f9c Mon Sep 17 00:00:00 2001 From: Artem Orobets Date: Fri, 17 Feb 2017 17:09:41 +0200 Subject: [PATCH 266/330] Docker image for secor --- Dockerfile | 9 +++ src/main/config/log4j.docker.properties | 8 +++ src/main/scripts/docker-entrypoint.sh | 76 +++++++++++++++++++++++++ 3 files changed, 93 insertions(+) create mode 100644 Dockerfile create mode 100644 src/main/config/log4j.docker.properties create mode 100644 src/main/scripts/docker-entrypoint.sh diff --git a/Dockerfile b/Dockerfile new file mode 100644 index 000000000..9a47a3b42 --- /dev/null +++ b/Dockerfile @@ -0,0 +1,9 @@ +FROM java: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/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/scripts/docker-entrypoint.sh b/src/main/scripts/docker-entrypoint.sh new file mode 100644 index 000000000..d9e8865ea --- /dev/null +++ b/src/main/scripts/docker-entrypoint.sh @@ -0,0 +1,76 @@ +#!/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 "$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_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 + +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=events-dev -Dlog4j.configuration=file:./${LOG4J_CONFIGURATION:-log4j.docker.properties} \ + -Dconfig=secor.prod.partition.properties $SECOR_CONFIG \ + -cp $CLASSPATH com.pinterest.secor.main.ConsumerMain From 73352a6099fd1e14f86ac84548f8405a8c73b96b Mon Sep 17 00:00:00 2001 From: serzh Date: Tue, 21 Feb 2017 16:26:41 +0200 Subject: [PATCH 267/330] Add case for transforming micors to millis --- .../com/pinterest/secor/parser/TimestampedMessageParser.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 722b381ca..8f7c6452f 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -128,10 +128,13 @@ static String usingMinutePrefix(SecorConfig config) { 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 From 47d73995f7f8aac2739138d7c6cd70fbc90d8af0 Mon Sep 17 00:00:00 2001 From: Dmytro Bochorishvili Date: Wed, 22 Feb 2017 15:05:43 +0200 Subject: [PATCH 268/330] Add SplitFieldMessageParser --- .../pinterest/secor/common/SecorConfig.java | 4 + .../parser/SplitByFieldMessageParser.java | 98 ++++++++++ .../parser/SplitByFieldMessageParserTest.java | 184 ++++++++++++++++++ 3 files changed, 286 insertions(+) create mode 100644 src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java create mode 100644 src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index c4ceff31b..37af2028c 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -389,6 +389,10 @@ 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); } 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..cba27b9b1 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java @@ -0,0 +1,98 @@ +/** + * 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 + { + // Partition finalization is not supported + return null; + } + + @Override + public String[] getPreviousPartitions(String[] partition) throws Exception + { + // Partition finalization is not supported + return null; + } + + 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/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java new file mode 100644 index 000000000..6bc2b3d92 --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java @@ -0,0 +1,184 @@ +/** + * 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; + + @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="); + + 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); + + 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); + + 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); + } + + @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 + public void testGetFinalizedUptoPartitions() throws Exception + { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + List lastMessages = new ArrayList(); + lastMessages.add(mMessageWithTypeAndTimestamp); + List committedMessages = new ArrayList(); + committedMessages.add(mMessageWithTypeAndTimestamp); + + String uptoPartitions[] = jsonMessageParser.getFinalizedUptoPartitions(lastMessages, + committedMessages); + assertNull(uptoPartitions); + + String previous[] = jsonMessageParser.getPreviousPartitions(uptoPartitions); + assertNull(previous); + } +} From 34f66572da5b702b75dc56d6ce692aa57165e26c Mon Sep 17 00:00:00 2001 From: Dmytro Bochorishvili Date: Fri, 24 Feb 2017 13:44:51 +0200 Subject: [PATCH 269/330] SplitByFieldMessageParser - Make partition finalization logic throw unsupported operation exception and fix code formatting --- .../parser/SplitByFieldMessageParser.java | 21 +++----- .../parser/SplitByFieldMessageParserTest.java | 50 ++++++++----------- 2 files changed, 29 insertions(+), 42 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java b/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java index cba27b9b1..9b76f84d2 100644 --- a/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java @@ -33,8 +33,7 @@ * * Caution: this parser doesn't support finalization of partitions. */ -public class SplitByFieldMessageParser extends TimestampedMessageParser implements Partitioner -{ +public class SplitByFieldMessageParser extends TimestampedMessageParser implements Partitioner { private static final Logger LOG = LoggerFactory.getLogger(SplitByFieldMessageParser.class); private final String mSplitFieldName; @@ -45,14 +44,12 @@ public SplitByFieldMessageParser(SecorConfig config) { } @Override - public long extractTimestampMillis(Message message) throws Exception - { + public long extractTimestampMillis(Message message) throws Exception { throw new UnsupportedOperationException("Unsupported, use extractPartitions method instead"); } @Override - public String[] extractPartitions(Message message) throws Exception - { + 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"); @@ -67,17 +64,13 @@ public String[] extractPartitions(Message message) throws Exception @Override public String[] getFinalizedUptoPartitions(List lastMessages, - List committedMessages) throws Exception - { - // Partition finalization is not supported - return null; + List committedMessages) throws Exception { + throw new UnsupportedOperationException("Partition finalization is not supported"); } @Override - public String[] getPreviousPartitions(String[] partition) throws Exception - { - // Partition finalization is not supported - return null; + public String[] getPreviousPartitions(String[] partitions) throws Exception { + throw new UnsupportedOperationException("Partition finalization is not supported"); } protected String extractEventType(JSONObject jsonObject) { diff --git a/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java index 6bc2b3d92..95259a535 100644 --- a/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java @@ -39,8 +39,7 @@ public class SplitByFieldMessageParserTest extends TestCase { private Message mMessageWithoutType; @Override - public void setUp() throws Exception - { + public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); Mockito.when(mConfig.getMessageSplitFieldName()).thenReturn("type"); Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); @@ -68,8 +67,7 @@ public void setUp() throws Exception } @Test - public void testExtractTypeAndTimestamp() throws Exception - { + public void testExtractTypeAndTimestamp() throws Exception { SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); assertEquals(1405911096000l, jsonMessageParser.extractTimestampMillis((JSONObject) JSONValue.parse(mMessageWithTypeAndTimestamp.getPayload()))); @@ -80,26 +78,23 @@ public void testExtractTypeAndTimestamp() throws Exception } @Test(expected = RuntimeException.class) - public void testExtractTimestampMillisExceptionNoTimestamp() throws Exception - { + 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 - { + @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 - { + @Test(expected = ClassCastException.class) + public void testExtractTimestampMillisException2() throws Exception { SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); byte emptyBytes2[] = "".getBytes(); @@ -107,8 +102,7 @@ public void testExtractTimestampMillisException2() throws Exception } @Test(expected = RuntimeException.class) - public void testExtractTimestampMillisExceptionNoType() throws Exception - { + public void testExtractTimestampMillisExceptionNoType() throws Exception { SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); // Throws exception if there's no timestamp, for any reason. @@ -116,8 +110,7 @@ public void testExtractTimestampMillisExceptionNoType() throws Exception } @Test - public void testExtractPartitions() throws Exception - { + public void testExtractPartitions() throws Exception { SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); String expectedEventTypePartition = "event1"; @@ -130,8 +123,7 @@ public void testExtractPartitions() throws Exception } @Test - public void testExtractHourlyPartitions() throws Exception - { + public void testExtractHourlyPartitions() throws Exception { Mockito.when(TimestampedMessageParser.usingHourly(mConfig)).thenReturn(true); SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); @@ -147,8 +139,7 @@ public void testExtractHourlyPartitions() throws Exception } @Test - public void testExtractHourlyPartitionsForNonUTCTimezone() throws Exception - { + 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); @@ -164,9 +155,8 @@ public void testExtractHourlyPartitionsForNonUTCTimezone() throws Exception assertEquals(expectedHrPartition, result[2]); } - @Test - public void testGetFinalizedUptoPartitions() throws Exception - { + @Test(expected = UnsupportedOperationException.class) + public void testGetFinalizedUptoPartitions() throws Exception { SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); List lastMessages = new ArrayList(); @@ -174,11 +164,15 @@ public void testGetFinalizedUptoPartitions() throws Exception List committedMessages = new ArrayList(); committedMessages.add(mMessageWithTypeAndTimestamp); - String uptoPartitions[] = jsonMessageParser.getFinalizedUptoPartitions(lastMessages, - committedMessages); - assertNull(uptoPartitions); + jsonMessageParser.getFinalizedUptoPartitions(lastMessages, committedMessages); + } - String previous[] = jsonMessageParser.getPreviousPartitions(uptoPartitions); - assertNull(previous); + @Test(expected = UnsupportedOperationException.class) + public void testGetPreviousPartitions() throws Exception { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + String partitions[] = {"event1", "dt=2014-07-21"}; + jsonMessageParser.getPreviousPartitions(partitions); } + } From 962f32f2a3c7d80de4a05dd6829ee716d6c63834 Mon Sep 17 00:00:00 2001 From: Ryan Smith-Evans Date: Tue, 28 Feb 2017 17:07:41 +0000 Subject: [PATCH 270/330] Added zookeeper path to docker --- src/main/scripts/docker-entrypoint.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/main/scripts/docker-entrypoint.sh b/src/main/scripts/docker-entrypoint.sh index d9e8865ea..9ac285d52 100644 --- a/src/main/scripts/docker-entrypoint.sh +++ b/src/main/scripts/docker-entrypoint.sh @@ -12,6 +12,14 @@ else 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" From 3aaeb92f259234793697d587a06c2d288ba2ac93 Mon Sep 17 00:00:00 2001 From: Ryan Smith-Evans Date: Wed, 1 Mar 2017 11:24:33 +0000 Subject: [PATCH 271/330] added message parser option to docker image --- src/main/scripts/docker-entrypoint.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/scripts/docker-entrypoint.sh b/src/main/scripts/docker-entrypoint.sh index 9ac285d52..70a9998b9 100644 --- a/src/main/scripts/docker-entrypoint.sh +++ b/src/main/scripts/docker-entrypoint.sh @@ -69,7 +69,10 @@ 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" From fb950db77d5bb5a41d020a0deb57fa31a0f732d1 Mon Sep 17 00:00:00 2001 From: Ryan Smith-Evans Date: Wed, 1 Mar 2017 15:46:20 +0000 Subject: [PATCH 272/330] More docker mappings to config --- src/main/scripts/docker-entrypoint.sh | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/main/scripts/docker-entrypoint.sh b/src/main/scripts/docker-entrypoint.sh index 70a9998b9..0ed1eb23c 100644 --- a/src/main/scripts/docker-entrypoint.sh +++ b/src/main/scripts/docker-entrypoint.sh @@ -35,6 +35,18 @@ if [[ ! -z "$SECOR_GROUP" ]]; then 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 @@ -51,6 +63,7 @@ if [[ ! -z "$S3_PATH" ]]; then 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" From 393b7e694e145be3d766d2731b714d21e9aac8b5 Mon Sep 17 00:00:00 2001 From: Prasanna Gautam Date: Wed, 1 Mar 2017 17:49:06 -0800 Subject: [PATCH 273/330] typo on transformation --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 6c7dc4581..3f846c7f2 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,7 @@ Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud Storage - **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 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 tranformation can be done by using customized class. + - **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 From a98e1bf3379b67f713d43e47cdaf3681ad2c9aa4 Mon Sep 17 00:00:00 2001 From: Ahsan Dar Date: Thu, 2 Mar 2017 11:58:59 +0800 Subject: [PATCH 274/330] fixed config name --- .../com/pinterest/secor/parser/TimestampedMessageParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index cf992bcbe..1ba15179b 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -123,7 +123,7 @@ static String usingHourPrefix(SecorConfig config) { } static String usingMinutePrefix(SecorConfig config) { - return config.getString("partitioner.granularity.min.prefix", "min="); + return config.getString("partitioner.granularity.minute.prefix", "min="); } protected static long toMillis(final long timestamp) { From 7141a4b0eb7804babab37bb1345087ba600d1fc6 Mon Sep 17 00:00:00 2001 From: Artem Orobets Date: Tue, 14 Mar 2017 12:27:28 +0200 Subject: [PATCH 275/330] Initialize secor_group JVM parameter with value of SECOR_GROUP environment variable. --- src/main/scripts/docker-entrypoint.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scripts/docker-entrypoint.sh b/src/main/scripts/docker-entrypoint.sh index 0ed1eb23c..5052cfb81 100644 --- a/src/main/scripts/docker-entrypoint.sh +++ b/src/main/scripts/docker-entrypoint.sh @@ -95,6 +95,6 @@ cd /opt/secor DEFAULT_CLASSPATH="*:lib/*" CLASSPATH=${CLASSPATH:-$DEFAULT_CLASSPATH} -java -Xmx${JVM_MEMORY:-512m} $JAVA_OPTS -ea -Dsecor_group=events-dev -Dlog4j.configuration=file:./${LOG4J_CONFIGURATION:-log4j.docker.properties} \ +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 From 418957020b7974509cc979fce82b43b6678a23d4 Mon Sep 17 00:00:00 2001 From: Weichu Liu Date: Fri, 17 Mar 2017 11:46:22 +0900 Subject: [PATCH 276/330] Bump log4j library to 1.2.17 to support EnhancedPatternLayout --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index cdd2aef12..f16fb6090 100644 --- a/pom.xml +++ b/pom.xml @@ -115,7 +115,7 @@ log4j log4j - 1.2.15 + 1.2.17 com.sun.jmx From 09d553a136e7cf8e22e65a33062e562b70e3d7ee Mon Sep 17 00:00:00 2001 From: Artem Orobets Date: Thu, 2 Mar 2017 14:43:58 +0200 Subject: [PATCH 277/330] MetricCollector component for ingesting secor performance metrics into a monitoring service. The component is pluggable, you can configure ingestion of metrics to any monitoring system by implementing MetricCollector interface. By default metrics are sent to Ostrich, statistic library bundled into secor. Metrics: - consumer.message_size_bytes - records distribution of message sizes processed by secor (min/max/avg/count) - consumer.throughput_bytes - total number of bytes processed by secor - consumer.message_errors.count - number of errors during message parsing - uploader.file_uploads.count - number of files uploaded --- src/main/config/secor.common.properties | 4 ++ .../pinterest/secor/common/SecorConfig.java | 4 ++ .../pinterest/secor/consumer/Consumer.java | 22 +++++--- .../secor/monitoring/MetricCollector.java | 47 +++++++++++++++++ .../monitoring/OstrichMetricCollector.java | 25 +++++++++ .../pinterest/secor/uploader/Uploader.java | 13 +++-- .../pinterest/secor/util/ReflectionUtil.java | 43 ++++++++++++--- .../config/secor.test.monitoring.properties | 1 + .../secor/common/SecorConfigTest.java | 20 +++++-- .../OstrichMetricCollectorTest.java | 52 +++++++++++++++++++ .../secor/uploader/UploaderTest.java | 3 +- .../secor/util/ReflectionUtilTest.java | 40 +++++++++++--- 12 files changed, 240 insertions(+), 34 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/monitoring/MetricCollector.java create mode 100644 src/main/java/com/pinterest/secor/monitoring/OstrichMetricCollector.java create mode 100644 src/test/config/secor.test.monitoring.properties create mode 100644 src/test/java/com/pinterest/secor/monitoring/OstrichMetricCollectorTest.java diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 4778d40ac..bcc05b9cb 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -377,3 +377,7 @@ 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 diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 37af2028c..0e802be40 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -565,4 +565,8 @@ public String[] getStringArray(String name) { public String getThriftProtocolClass() { return mProperties.getString("secor.thrift.protocol.class"); } + + public String getMetricsCollectorClass() { + return mProperties.getString("secor.monitoring.metrics.collector.class"); + } } diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 9bb83f951..8ec3dbe4e 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -21,21 +21,19 @@ 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.reader.MessageReader; import com.pinterest.secor.transformer.MessageTransformer; -import com.pinterest.secor.uploader.Uploader; import com.pinterest.secor.uploader.UploadManager; -import com.pinterest.secor.reader.MessageReader; +import com.pinterest.secor.uploader.Uploader; import com.pinterest.secor.util.ReflectionUtil; import com.pinterest.secor.writer.MessageWriter; - import kafka.consumer.ConsumerTimeoutException; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.lang.Thread; /** * Consumer is a top-level component coordinating reading, writing, and uploading Kafka log @@ -51,6 +49,7 @@ public class Consumer extends Thread { private static final Logger LOG = LoggerFactory.getLogger(Consumer.class); protected SecorConfig mConfig; + protected MetricCollector mMetricCollector; protected MessageReader mMessageReader; protected MessageWriter mMessageWriter; @@ -68,14 +67,16 @@ public Consumer(SecorConfig config) { private void init() throws Exception { 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); + 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); + mMessageTransformer = ReflectionUtil.createMessageTransformer(mConfig.getMessageTransformerClass(), mConfig); mUnparsableMessages = 0.; } @@ -145,6 +146,8 @@ protected boolean consumeNextMessage() { 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) { @@ -156,6 +159,9 @@ protected boolean consumeNextMessage() { if (parsedMessage != null) { try { 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) { throw new RuntimeException("Failed to write message " + parsedMessage, e); } @@ -166,7 +172,7 @@ protected boolean consumeNextMessage() { /** * Helper to get the offset tracker (used in tests) - * + * * @return the offset tracker */ public OffsetTracker getOffsetTracker() { 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/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index d923d444c..0c9048743 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -21,6 +21,7 @@ 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; @@ -43,6 +44,7 @@ public class Uploader { private static final Logger LOG = LoggerFactory.getLogger(Uploader.class); protected SecorConfig mConfig; + protected MetricCollector mMetricCollector; protected OffsetTracker mOffsetTracker; protected FileRegistry mFileRegistry; protected ZookeeperConnector mZookeeperConnector; @@ -57,24 +59,25 @@ public class Uploader { * @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) { + UploadManager uploadManager, MetricCollector metricCollector) { init(config, offsetTracker, fileRegistry, uploadManager, - new ZookeeperConnector(config)); + new ZookeeperConnector(config), metricCollector); } // For testing use only. public void init(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, UploadManager uploadManager, - ZookeeperConnector zookeeperConnector) { + ZookeeperConnector zookeeperConnector, MetricCollector metricCollector) { mConfig = config; mOffsetTracker = offsetTracker; mFileRegistry = fileRegistry; mUploadManager = uploadManager; mZookeeperConnector = zookeeperConnector; mTopicFilter = mConfig.getKafkaTopicUploadAtMinuteMarkFilter(); - + mMetricCollector = metricCollector; } protected void uploadFiles(TopicPartition topicPartition) throws Exception { @@ -110,6 +113,8 @@ protected void uploadFiles(TopicPartition topicPartition) throws Exception { 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); diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 76383463c..208785a04 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,17 +16,17 @@ */ package com.pinterest.secor.util; -import org.apache.hadoop.io.compress.CompressionCodec; - 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 org.apache.hadoop.io.compress.CompressionCodec; /** * ReflectionUtil implements utility methods to construct objects of classes @@ -118,7 +118,7 @@ public static MessageParser createMessageParser(String className, * @throws Exception */ private static FileReaderWriterFactory createFileReaderWriterFactory(String className, - SecorConfig config) throws Exception { + SecorConfig config) throws Exception { Class clazz = Class.forName(className); if (!FileReaderWriterFactory.class.isAssignableFrom(clazz)) { throw new IllegalArgumentException(String.format("The class '%s' is not assignable to '%s'.", @@ -168,7 +168,7 @@ public static FileReader createFileReader(String className, LogFilePath logFileP 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 @@ -177,7 +177,7 @@ public static FileReader createFileReader(String className, LogFilePath logFileP * config. * * See the secor.message.transformer.class config option. - * + * * @param className * @param config * @return @@ -194,4 +194,31 @@ public static MessageTransformer createMessageTransformer( return (MessageTransformer) clazz.getConstructor(SecorConfig.class) .newInstance(config); } + + /** + * 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())); + } + + return (MetricCollector) clazz.newInstance(); + } } 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/java/com/pinterest/secor/common/SecorConfigTest.java b/src/test/java/com/pinterest/secor/common/SecorConfigTest.java index b8d507aa3..6e2e7f834 100644 --- a/src/test/java/com/pinterest/secor/common/SecorConfigTest.java +++ b/src/test/java/com/pinterest/secor/common/SecorConfigTest.java @@ -1,16 +1,15 @@ 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 com.pinterest.secor.protobuf.Messages.UnitTestMessage1; -import com.pinterest.secor.protobuf.Messages.UnitTestMessage2; - import java.net.URL; import java.util.Map; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; public class SecorConfigTest { @@ -48,5 +47,16 @@ public void testProtobufMessageClassPerTopic() throws ConfigurationException { 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/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/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index 74e929660..4ae220ccf 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -20,6 +20,7 @@ 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; @@ -55,7 +56,7 @@ public TestUploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, UploadManager uploadManager, ZookeeperConnector zookeeperConnector) { - init(config, offsetTracker, fileRegistry, uploadManager, zookeeperConnector); + init(config, offsetTracker, fileRegistry, uploadManager, zookeeperConnector, Mockito.mock(MetricCollector.class)); mReader = Mockito.mock(FileReader.class); } diff --git a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java index 37f1dcec4..8ad8c2b4e 100644 --- a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java +++ b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.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,10 +18,14 @@ 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.junit.Test; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; import org.junit.Before; +import org.junit.Test; public class ReflectionUtilTest { @@ -31,12 +35,12 @@ public class ReflectionUtilTest { @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="); + 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 @@ -70,4 +74,24 @@ public void testFileWriterConstructorMissing() throws Exception { 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"); + } } From 458ab80567c30c6e855b4f3f0c4dbdae4f0b9e47 Mon Sep 17 00:00:00 2001 From: max thomas Date: Wed, 5 Apr 2017 19:00:38 -0500 Subject: [PATCH 278/330] Updates the 0.10 kafka client to the latest version. Per the release notes, this client is capable of detecting any kafka versions between 0.10.0.x and 0.10.2.x. This change allows users of intermediate kafka versions (e.g. 0.10.1 or 0.10.2) to use secor without any other changes. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f16fb6090..da0ec61e7 100644 --- a/pom.xml +++ b/pom.xml @@ -620,7 +620,7 @@ org.apache.kafka kafka_2.10 - 0.10.0.1 + 0.10.2.0 org.slf4j From ffc69e7a02e3c7f71126204b60fdfbc94827ac17 Mon Sep 17 00:00:00 2001 From: Gabe Frangakis Date: Mon, 17 Apr 2017 14:07:34 -0400 Subject: [PATCH 279/330] bump parquet version, expose configuration variables --- pom.xml | 2 +- src/main/config/secor.common.properties | 17 ++++++++++++++ ...rotobufParquetFileReaderWriterFactory.java | 14 ++++++++++-- .../ThriftParquetFileReaderWriterFactory.java | 14 +++++++++++- .../com/pinterest/secor/util/ParquetUtil.java | 22 +++++++++++++++++++ ...bufParquetFileReaderWriterFactoryTest.java | 13 +++++++++++ ...iftParquetFileReaderWriterFactoryTest.java | 14 +++++++++++- 7 files changed, 91 insertions(+), 5 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/util/ParquetUtil.java diff --git a/pom.xml b/pom.xml index da0ec61e7..8da00b4af 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ 1.6 UTF-8 UTF-8 - 1.8.1 + 1.9.0 diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index bcc05b9cb..f90437f71 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -381,3 +381,20 @@ 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 diff --git a/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java index 9129a68b1..d3d3309a0 100644 --- a/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java @@ -5,7 +5,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.parquet.hadoop.ParquetReader; -import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.proto.ProtoParquetReader; import org.apache.parquet.proto.ProtoParquetWriter; @@ -19,6 +18,7 @@ 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; /** @@ -30,8 +30,18 @@ public class ProtobufParquetFileReaderWriterFactory implements FileReaderWriterF 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 @@ -81,7 +91,7 @@ public ProtobufParquetFileWriter(LogFilePath logFilePath, CompressionCodec codec .fromCompressionCodec(codec != null ? codec.getClass() : null); topic = logFilePath.getTopic(); writer = new ProtoParquetWriter(path, protobufUtil.getMessageClass(topic), codecName, - ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE); + blockSize, pageSize, enableDictionary, validating); } @Override diff --git a/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java index ca9ed9d4c..21fd4b295 100644 --- a/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java @@ -17,6 +17,7 @@ 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; /** @@ -30,8 +31,18 @@ public class ThriftParquetFileReaderWriterFactory implements FileReaderWriterFac 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 @@ -92,7 +103,8 @@ public ThriftParquetFileWriter(LogFilePath logFilePath, CompressionCodec codec) 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); + writer = new ThriftParquetWriter(path, thriftUtil.getMessageClass(topic), codecName, + blockSize, pageSize, enableDictionary, validating); } @Override 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/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java index 19f2b4892..8aa4849f4 100644 --- a/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java +++ b/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java @@ -21,6 +21,8 @@ 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; @@ -33,8 +35,10 @@ 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) @@ -54,6 +58,15 @@ public void testProtobufParquetReadWriteRoundTrip() throws Exception { 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"); diff --git a/src/test/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactoryTest.java index 6f854b9ef..dcfe8c1f4 100644 --- a/src/test/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactoryTest.java +++ b/src/test/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactoryTest.java @@ -21,6 +21,8 @@ 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; @@ -36,6 +38,7 @@ 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; @@ -58,7 +61,16 @@ public void testThriftParquetReadWriteRoundTrip() throws Exception { Mockito.when(config.getFileReaderWriterFactory()) .thenReturn(ThriftParquetFileReaderWriterFactory.class.getName()); Mockito.when(config.getThriftProtocolClass()) - .thenReturn(TCompactProtocol.class.getName()); + .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"); From 80d05f92d214de60047bfd4988a42fe094bf6072 Mon Sep 17 00:00:00 2001 From: jaimess Date: Tue, 9 May 2017 13:00:03 +0200 Subject: [PATCH 280/330] Update ThriftParquetFileReaderWriterFactory.java --- .../secor/io/impl/ThriftParquetFileReaderWriterFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java index 21fd4b295..2ff16024d 100644 --- a/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java @@ -25,7 +25,7 @@ * com.pinterest.secor.io.impl.ProtobufParquetFileReaderWriterFactory * Implementation for reading/writing thrift messages to/from Parquet files. * - * @author jaime sastre (jaime.sastre.s@gmail.com) + */ public class ThriftParquetFileReaderWriterFactory implements FileReaderWriterFactory { From 6fd65b2147be959be34525f6cb056ccbccc1d2b4 Mon Sep 17 00:00:00 2001 From: Ashish Kumar Date: Tue, 9 May 2017 18:17:39 +0530 Subject: [PATCH 281/330] Orc reader/writer added --- pom.xml | 5 + src/main/config/secor.common.properties | 8 +- .../pinterest/secor/common/SecorConfig.java | 15 ++ .../impl/JsonORCFileReaderWriterFactory.java | 181 ++++++++++++++ .../secor/util/orc/JsonFieldFiller.java | 126 ++++++++++ .../secor/util/orc/VectorColumnFiller.java | 227 ++++++++++++++++++ .../orc/schema/DefaultORCSchemaProvider.java | 52 ++++ .../util/orc/schema/ORCScehmaProvider.java | 22 ++ 8 files changed, 635 insertions(+), 1 deletion(-) create mode 100644 src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java create mode 100644 src/main/java/com/pinterest/secor/util/orc/JsonFieldFiller.java create mode 100644 src/main/java/com/pinterest/secor/util/orc/VectorColumnFiller.java create mode 100644 src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java create mode 100644 src/main/java/com/pinterest/secor/util/orc/schema/ORCScehmaProvider.java diff --git a/pom.xml b/pom.xml index 8da00b4af..22e56be17 100644 --- a/pom.xml +++ b/pom.xml @@ -310,6 +310,11 @@ azure-storage 4.0.0 + + org.apache.orc + orc-core + 1.3.3 + diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index f90437f71..c1b754593 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -51,7 +51,7 @@ aws.proxy.http.port= # secor.upload.manager.class. # # http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region -aws.region= +aws.region=SG aws.endpoint= # Toggle the AWS S3 client between virtual host style access and path style @@ -398,3 +398,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> +# Individual topic schema: +secor.orc.message.schema.topic1=struct\,f:array\,g:int> \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 0e802be40..513a50bc6 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -569,4 +569,19 @@ public String getThriftProtocolClass() { public String getMetricsCollectorClass() { return mProperties.getString("secor.monitoring.metrics.collector.class"); } + + 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"); + } } 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..5010912c7 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java @@ -0,0 +1,181 @@ +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 com.google.gson.Gson; +import com.google.gson.JsonObject; +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.orc.JsonFieldFiller; +import com.pinterest.secor.util.orc.VectorColumnFiller; +import com.pinterest.secor.util.orc.VectorColumnFiller.JsonConverter; +import com.pinterest.secor.util.orc.schema.DefaultORCSchemaProvider; +import com.pinterest.secor.util.orc.schema.ORCScehmaProvider; + +/** + * ORC reader/writer implementation + * + * @author Ashish (ashu.impetus@gmail.com) + * + */ +public class JsonORCFileReaderWriterFactory implements FileReaderWriterFactory { + + private ORCScehmaProvider schemaProvider; + + public JsonORCFileReaderWriterFactory(SecorConfig config) { + schemaProvider = new DefaultORCSchemaProvider(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()); + 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) { + e.printStackTrace(); + } + 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()); + 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/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..ea49203a6 --- /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[batch.size] = 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..acb734e2c --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java @@ -0,0 +1,52 @@ +package com.pinterest.secor.util.orc.schema; + +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.orc.TypeDescription; + +import com.google.common.collect.Maps; +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 = Maps.newHashMap(); + setSchemas(config); + } + + @Override + public TypeDescription getSchema(String topic) { + return topicToSchemaMap.getOrDefault(topic, 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..5f5268fb5 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/orc/schema/ORCScehmaProvider.java @@ -0,0 +1,22 @@ +package com.pinterest.secor.util.orc.schema; + +import org.apache.orc.TypeDescription; + +/** + * 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 + * @return + */ + public TypeDescription getSchema(String topic); + +} From 86467b43f9ba9cad48a585e789ec78381031730d Mon Sep 17 00:00:00 2001 From: Ashish Kumar Date: Thu, 11 May 2017 14:39:38 +0530 Subject: [PATCH 282/330] Added support for custom ORC-schema-provider implementation --- src/main/config/secor.common.properties | 6 ++-- .../pinterest/secor/common/SecorConfig.java | 4 +++ .../impl/JsonORCFileReaderWriterFactory.java | 13 +++++---- .../pinterest/secor/util/ReflectionUtil.java | 28 +++++++++++++++++++ .../orc/schema/DefaultORCSchemaProvider.java | 3 +- .../util/orc/schema/ORCScehmaProvider.java | 7 +++-- 6 files changed, 50 insertions(+), 11 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index c1b754593..1da2de783 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -51,7 +51,7 @@ aws.proxy.http.port= # secor.upload.manager.class. # # http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region -aws.region=SG +aws.region= aws.endpoint= # Toggle the AWS S3 client between virtual host style access and path style @@ -402,5 +402,5 @@ 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> -# Individual topic schema: -secor.orc.message.schema.topic1=struct\,f:array\,g:int> \ No newline at end of file +# 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 \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 513a50bc6..17e1a095c 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -584,4 +584,8 @@ public Map getPropertyMapForPrefix(String prefix) { 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/io/impl/JsonORCFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java index 5010912c7..46c0de918 100644 --- a/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java @@ -28,10 +28,10 @@ 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.DefaultORCSchemaProvider; import com.pinterest.secor.util.orc.schema.ORCScehmaProvider; /** @@ -44,8 +44,9 @@ public class JsonORCFileReaderWriterFactory implements FileReaderWriterFactory { private ORCScehmaProvider schemaProvider; - public JsonORCFileReaderWriterFactory(SecorConfig config) { - schemaProvider = new DefaultORCSchemaProvider(config); + public JsonORCFileReaderWriterFactory(SecorConfig config) throws Exception { + schemaProvider = ReflectionUtil.createORCSchemaProvider( + config.getORCSchemaProviderClass(), config); } @Override @@ -71,7 +72,8 @@ protected class JsonORCFileReader implements FileReader { @SuppressWarnings("deprecation") public JsonORCFileReader(LogFilePath logFilePath, CompressionCodec codec) throws IOException { - schema = schemaProvider.getSchema(logFilePath.getTopic()); + schema = schemaProvider.getSchema(logFilePath.getTopic(), + logFilePath); Path path = new Path(logFilePath.getLogFilePath()); Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(new Configuration(true))); @@ -125,7 +127,8 @@ public JsonORCFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { Configuration conf = new Configuration(); Path path = new Path(logFilePath.getLogFilePath()); - schema = schemaProvider.getSchema(logFilePath.getTopic()); + schema = schemaProvider.getSchema(logFilePath.getTopic(), + logFilePath); List fieldTypes = schema.getChildren(); converters = new JsonConverter[fieldTypes.size()]; for (int c = 0; c < converters.length; ++c) { diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 208785a04..8bcfbd787 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -26,6 +26,8 @@ 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 org.apache.hadoop.io.compress.CompressionCodec; /** @@ -221,4 +223,30 @@ public static MetricCollector createMetricCollector(String className) 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())); + } + return (ORCScehmaProvider) clazz.getConstructor(SecorConfig.class) + .newInstance(config); + } } 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 index acb734e2c..c7dc5a478 100644 --- a/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java +++ b/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java @@ -6,6 +6,7 @@ import org.apache.orc.TypeDescription; import com.google.common.collect.Maps; +import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.common.SecorConfig; /** @@ -27,7 +28,7 @@ public DefaultORCSchemaProvider(SecorConfig config) { } @Override - public TypeDescription getSchema(String topic) { + public TypeDescription getSchema(String topic, LogFilePath logFilePath) { return topicToSchemaMap.getOrDefault(topic, schemaForAlltopic); } 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 index 5f5268fb5..cd96d1c07 100644 --- a/src/main/java/com/pinterest/secor/util/orc/schema/ORCScehmaProvider.java +++ b/src/main/java/com/pinterest/secor/util/orc/schema/ORCScehmaProvider.java @@ -2,6 +2,8 @@ import org.apache.orc.TypeDescription; +import com.pinterest.secor.common.LogFilePath; + /** * ORC schema provider interface * @@ -14,9 +16,10 @@ 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 + * @param topic kafka topic + * @param logFilePath It may require to figure out the schema * @return */ - public TypeDescription getSchema(String topic); + public TypeDescription getSchema(String topic, LogFilePath logFilePath); } From 068b84b418e3be1fcdc9bc13738f4148f5b9a9c7 Mon Sep 17 00:00:00 2001 From: Ashish Kumar Date: Fri, 12 May 2017 13:30:17 +0530 Subject: [PATCH 283/330] Logger added in case of json parsing exception --- .../secor/io/impl/JsonORCFileReaderWriterFactory.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java index 46c0de918..ac9bbf0e3 100644 --- a/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java @@ -19,9 +19,12 @@ 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; @@ -42,6 +45,7 @@ */ public class JsonORCFileReaderWriterFactory implements FileReaderWriterFactory { + private static final Logger LOG = LoggerFactory.getLogger(FileRegistry.class); private ORCScehmaProvider schemaProvider; public JsonORCFileReaderWriterFactory(SecorConfig config) throws Exception { @@ -102,7 +106,7 @@ public KeyValue next() throws IOException { JsonFieldFiller.processRow(new JSONWriter(sw), batch, schema, rowIndex); } catch (JSONException e) { - e.printStackTrace(); + LOG.error("Unable to parse json {}", sw.toString()); } rowIndex++; return new KeyValue(offset++, sw.toString().getBytes("UTF-8")); From 73d0ace0fe0f16ad4c162746cec11351cfdb0158 Mon Sep 17 00:00:00 2001 From: Ahsan Nabi Dar Date: Fri, 12 May 2017 16:11:17 +0800 Subject: [PATCH 284/330] Update README.md --- README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 3f846c7f2..8297dbc33 100644 --- a/README.md +++ b/README.md @@ -153,8 +153,8 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [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) + * [Lovenish Goyal](https://github.com/lovenishgoyal) + * [Ahsan Nabi Dar](https://github.com/ahsandar) ## Companies who use Secor @@ -170,7 +170,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Zalando](http://www.zalando.com) * [Rakuten](http://techblog.rakuten.co.jp/) * [Appsflyer](https://www.appsflyer.com) - * [Wego] (http://www.wego.com) + * [Wego](http://www.wego.com) ## Help From 726aa37c1ba8d99d05d152994dcb92b0781bf451 Mon Sep 17 00:00:00 2001 From: Ashish Kumar Date: Fri, 12 May 2017 14:05:36 +0530 Subject: [PATCH 285/330] Google Maps dependency removed --- .../secor/util/orc/schema/DefaultORCSchemaProvider.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 index c7dc5a478..ac1db8007 100644 --- a/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java +++ b/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java @@ -1,11 +1,11 @@ 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.google.common.collect.Maps; import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.common.SecorConfig; @@ -23,7 +23,7 @@ public class DefaultORCSchemaProvider implements ORCScehmaProvider { private TypeDescription schemaForAlltopic; public DefaultORCSchemaProvider(SecorConfig config) { - topicToSchemaMap = Maps.newHashMap(); + topicToSchemaMap = new HashMap(); setSchemas(config); } From 7dbc35670d836a961ef96baee64e4e4ece2d17a3 Mon Sep 17 00:00:00 2001 From: Ashish Kumar Date: Fri, 12 May 2017 14:17:17 +0530 Subject: [PATCH 286/330] Java 7 issue fixed --- .../secor/util/orc/schema/DefaultORCSchemaProvider.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 index ac1db8007..7696725c8 100644 --- a/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java +++ b/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java @@ -29,7 +29,11 @@ public DefaultORCSchemaProvider(SecorConfig config) { @Override public TypeDescription getSchema(String topic, LogFilePath logFilePath) { - return topicToSchemaMap.getOrDefault(topic, schemaForAlltopic); + TypeDescription topicSpecificTD = topicToSchemaMap.get(topic); + if (null != topicSpecificTD) { + return topicSpecificTD; + } + return schemaForAlltopic; } /** From d9f3de3461e86d56ebf28199de64c4fe809185e5 Mon Sep 17 00:00:00 2001 From: Ahsan Nabi Dar Date: Fri, 12 May 2017 16:55:40 +0800 Subject: [PATCH 287/330] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 8297dbc33..e2cea6f64 100644 --- a/README.md +++ b/README.md @@ -170,7 +170,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Zalando](http://www.zalando.com) * [Rakuten](http://techblog.rakuten.co.jp/) * [Appsflyer](https://www.appsflyer.com) - * [Wego](http://www.wego.com) + * [Wego](https://www.wego.com) ## Help From 1e37af490f33cae6fa1d19c0b543ab204416f853 Mon Sep 17 00:00:00 2001 From: Ashish Kumar Date: Sat, 13 May 2017 00:37:23 +0530 Subject: [PATCH 288/330] Doc added --- README.md | 4 ++++ src/main/config/secor.common.properties | 2 +- .../com/pinterest/secor/common/SecorConfig.java | 15 +++++++++++++++ .../io/impl/JsonORCFileReaderWriterFactory.java | 1 + 4 files changed, 21 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 3f846c7f2..515026a5d 100644 --- a/README.md +++ b/README.md @@ -90,6 +90,9 @@ Currently secor supports the following output formats - **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`. @@ -155,6 +158,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [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) ## Companies who use Secor diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 1da2de783..857c2eadc 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -403,4 +403,4 @@ parquet.validation=false # 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 \ No newline at end of file +secor.orc.schema.provider=com.pinterest.secor.util.orc.schema.DefaultORCSchemaProvider diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 17e1a095c..f436b8c08 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -570,6 +570,21 @@ public String getMetricsCollectorClass() { return mProperties.getString("secor.monitoring.metrics.collector.class"); } + /** + * 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(); diff --git a/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java index ac9bbf0e3..62359b7d1 100644 --- a/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java @@ -107,6 +107,7 @@ public KeyValue next() throws IOException { rowIndex); } catch (JSONException e) { LOG.error("Unable to parse json {}", sw.toString()); + return null; } rowIndex++; return new KeyValue(offset++, sw.toString().getBytes("UTF-8")); From a9a57c128c1ce6ec7b80c85880db536f1e6c3946 Mon Sep 17 00:00:00 2001 From: tygrash Date: Tue, 16 May 2017 17:53:36 +0530 Subject: [PATCH 289/330] Upgraded kafka version with new tag --- README.md | 2 ++ pom.xml | 8 ++++---- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 7b35cd2cd..eddeb0ec2 100644 --- a/README.md +++ b/README.md @@ -158,6 +158,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [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 @@ -175,6 +176,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Rakuten](http://techblog.rakuten.co.jp/) * [Appsflyer](https://www.appsflyer.com) * [Wego](https://www.wego.com) + * [GO_JEK](http://gojekengineering.com/) ## Help diff --git a/pom.xml b/pom.xml index 22e56be17..267be9906 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.23-SNAPSHOT + 0.24-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter @@ -602,9 +602,6 @@ kafka-0.8-dev - - true - org.apache.kafka @@ -621,6 +618,9 @@ kafka-0.10-dev + + true + org.apache.kafka From 12ac02b84a24eee7c03ac0efd4b966cbcb5ecec8 Mon Sep 17 00:00:00 2001 From: tygrash Date: Tue, 16 May 2017 23:21:58 +0530 Subject: [PATCH 290/330] Reverting change of version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 267be9906..baf42fab8 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.24-SNAPSHOT + 0.23-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 33e3029c0d999837b412598d61f843ff326b65b7 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 16 May 2017 14:27:09 -0700 Subject: [PATCH 291/330] Bump up the version to 0.23 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index baf42fab8..1578a9ac3 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.23-SNAPSHOT + 0.23 jar secor Kafka to s3/gs/swift logs exporter From 872a1eea0266b7155c5d5c73900a22516945e24e Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Tue, 16 May 2017 14:33:52 -0700 Subject: [PATCH 292/330] Bump up the version to 0.24-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1578a9ac3..267be9906 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.23 + 0.24-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From b6f11d17c2e4119c5bd0dcaa3a63938fe2e2b36e Mon Sep 17 00:00:00 2001 From: tygrash Date: Wed, 17 May 2017 11:08:57 +0530 Subject: [PATCH 293/330] Added timestamp in Message class and populating it while extracting other fields from kafka message --- README.md | 2 +- .../pinterest/secor/common/KafkaClient.java | 5 ++- .../com/pinterest/secor/message/Message.java | 11 +++++-- .../secor/message/ParsedMessage.java | 4 +-- .../pinterest/secor/parser/MessageParser.java | 6 ++-- .../pinterest/secor/reader/MessageReader.java | 2 +- .../secor/common/LogFilePathTest.java | 4 ++- .../pinterest/secor/message/MessageTest.java | 4 +-- .../secor/parser/DateMessageParserTest.java | 27 ++++++++-------- .../secor/parser/Iso8601ParserTest.java | 29 +++++++++-------- .../secor/parser/JsonMessageParserTest.java | 31 ++++++++++--------- .../secor/parser/MessagePackParserTest.java | 19 ++++++------ .../parser/ProtobufMessageParserTest.java | 13 +++++--- .../parser/ProtobufTimestampParserTest.java | 13 +++++--- .../secor/parser/RegexMessageParserTest.java | 9 ++++-- .../parser/SplitByFieldMessageParserTest.java | 9 ++++-- .../secor/parser/ThriftMessageParserTest.java | 5 ++- 17 files changed, 109 insertions(+), 84 deletions(-) diff --git a/README.md b/README.md index eddeb0ec2..275aae312 100644 --- a/README.md +++ b/README.md @@ -176,7 +176,7 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l * [Rakuten](http://techblog.rakuten.co.jp/) * [Appsflyer](https://www.appsflyer.com) * [Wego](https://www.wego.com) - * [GO_JEK](http://gojekengineering.com/) + * [GO-JEK](http://gojekengineering.com/) ## Help diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 6da05cc69..c6971fc58 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -141,8 +141,11 @@ private Message getMessage(TopicPartition topicPartition, long offset, payloadBytes = new byte[payload.limit()]; payload.get(payloadBytes); } + Long timestamp = null; + timestamp = messageAndOffset.message().timestamp(); + return new Message(topicPartition.getTopic(), topicPartition.getPartition(), - messageAndOffset.offset(), keyBytes, payloadBytes); + messageAndOffset.offset(), keyBytes, payloadBytes, timestamp); } private SimpleConsumer createConsumer(String host, int port, String clientName) { diff --git a/src/main/java/com/pinterest/secor/message/Message.java b/src/main/java/com/pinterest/secor/message/Message.java index 1b51b6063..384336313 100644 --- a/src/main/java/com/pinterest/secor/message/Message.java +++ b/src/main/java/com/pinterest/secor/message/Message.java @@ -34,13 +34,15 @@ public class Message { private long mOffset; private byte[] mKafkaKey; private byte[] mPayload; + private Long mTimestamp; protected String fieldsToString() { return "topic='" + mTopic + '\'' + ", kafkaPartition=" + mKafkaPartition + ", offset=" + mOffset + ", kafkaKey=" + new String(mKafkaKey) + - ", payload=" + new String(mPayload); + ", payload=" + new String(mPayload) + + ", timestamp=" + mTimestamp; } @Override @@ -48,7 +50,7 @@ public String toString() { return "Message{" + fieldsToString() + '}'; } - public Message(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload) { + public Message(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload, Long timestamp) { mTopic = topic; mKafkaPartition = kafkaPartition; mOffset = offset; @@ -60,6 +62,7 @@ public Message(String topic, int kafkaPartition, long offset, byte[] kafkaKey, b if (mPayload == null) { mPayload = EMPTY_BYTES; } + mTimestamp = timestamp; } public String getTopic() { @@ -82,6 +85,10 @@ 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 6e212df5c..d3d0bf65e 100644 --- a/src/main/java/com/pinterest/secor/message/ParsedMessage.java +++ b/src/main/java/com/pinterest/secor/message/ParsedMessage.java @@ -35,8 +35,8 @@ public String toString() { } public ParsedMessage(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload, - String[] mPartitions) { - super(topic, kafkaPartition, offset, kafkaKey, payload); + String[] mPartitions, long timestamp) { + super(topic, kafkaPartition, offset, kafkaKey, payload, timestamp); this.mPartitions = mPartitions; } diff --git a/src/main/java/com/pinterest/secor/parser/MessageParser.java b/src/main/java/com/pinterest/secor/parser/MessageParser.java index da10b7016..f21e4eeb3 100644 --- a/src/main/java/com/pinterest/secor/parser/MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/MessageParser.java @@ -19,10 +19,10 @@ 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 net.minidev.json.JSONObject; -import net.minidev.json.JSONValue; + import java.util.regex.Pattern; // TODO(pawel): should we offer a multi-message parser capable of parsing multiple types of @@ -60,7 +60,7 @@ public ParsedMessage parse(Message message) throws Exception { String[] partitions = extractPartitions(message); return new ParsedMessage(message.getTopic(), message.getKafkaPartition(), message.getOffset(), message.getKafkaKey(), - message.getPayload(), partitions); + message.getPayload(), partitions, message.getTimestamp()); } public abstract String[] extractPartitions(Message payload) throws Exception; diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 4efcb6512..871920c89 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -162,7 +162,7 @@ public Message read() { MessageAndMetadata kafkaMessage = mIterator.next(); Message message = new Message(kafkaMessage.topic(), kafkaMessage.partition(), kafkaMessage.offset(), kafkaMessage.key(), - kafkaMessage.message()); + kafkaMessage.message(), kafkaMessage.timestamp()); TopicPartition topicPartition = new TopicPartition(message.getTopic(), message.getKafkaPartition()); updateAccessTime(topicPartition); diff --git a/src/test/java/com/pinterest/secor/common/LogFilePathTest.java b/src/test/java/com/pinterest/secor/common/LogFilePathTest.java index 9af8887e2..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, null, - "some_payload".getBytes(), PARTITIONS); + "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/message/MessageTest.java b/src/test/java/com/pinterest/secor/message/MessageTest.java index c02ffaea7..50191b02a 100644 --- a/src/test/java/com/pinterest/secor/message/MessageTest.java +++ b/src/test/java/com/pinterest/secor/message/MessageTest.java @@ -1,14 +1,12 @@ package com.pinterest.secor.message; -import static org.junit.Assert.*; - import org.junit.Test; public class MessageTest { @Test public void testNullPayload() { - Message message = new Message("testTopic", 0, 123, null, null); + Message message = new Message("testTopic", 0, 123, null, null, null); System.out.println(message); // no assert necessary, just making sure it does not throw a diff --git a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index 373489bed..7e9f2225a 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -16,18 +16,15 @@ */ package com.pinterest.secor.parser; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; import junit.framework.TestCase; - -import java.util.TimeZone; - 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 { @@ -40,40 +37,42 @@ public class DateMessageParserTest extends TestCase { private Message mISOFormat; private Message mNanosecondISOFormat; private Message mNestedISOFormat; - private OngoingStubbing getTimestamp; + private long timestamp; @Override public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); 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, null, format1); + 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, null, format2); + 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, null, format3); + 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, null, invalidDate); + 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); + 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); + 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); + mNestedISOFormat = new Message("test", 0, 0, null, nestedISOFormat, timestamp); } @Test diff --git a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java index e3e412e62..1bd1dc056 100644 --- a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java @@ -16,18 +16,15 @@ */ package com.pinterest.secor.parser; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; import junit.framework.TestCase; - -import java.util.TimeZone; - 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 Iso8601ParserTest extends TestCase { @@ -41,44 +38,46 @@ public class Iso8601ParserTest extends TestCase { private Message mNestedISOFormat; private Message mNanosecondISOFormat; private Message mMissingDate; - private OngoingStubbing getTimestamp; + private long timestamp; @Override public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); + 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); + 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); + 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); + 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); + 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); + 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); + 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); + 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); + mMissingDate = new Message("test", 0, 0, null, missingDate, timestamp); } @Test diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index 964287d99..e2e5d8419 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -38,6 +38,7 @@ public class JsonMessageParserTest extends TestCase { private Message mMessageWithMillisFloatTimestamp; private Message mMessageWithoutTimestamp; private Message mMessageWithNestedTimestamp; + private long timestamp; @Override public void setUp() throws Exception { @@ -46,32 +47,34 @@ public void setUp() throws Exception { 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="); + 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 = 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); + mMessageWithSecondsTimestamp = new Message("test", 0, 0, null, messageWithSecondsTimestamp, timestamp); byte 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); + mMessageWithMillisTimestamp = new Message("test", 0, 0, null, messageWithMillisTimestamp, timestamp); byte 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); + 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, null, messageWithoutTimestamp); + mMessageWithoutTimestamp = new Message("test", 0, 0, null, messageWithoutTimestamp, timestamp); 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); + mMessageWithNestedTimestamp = new Message("test", 0, 0, null, messageWithNestedTimestamp, timestamp); } @Test @@ -101,7 +104,7 @@ public void testExtractTimestampMillisException1() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); byte emptyBytes1[] = {}; - jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes1)); + jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes1, timestamp)); } @Test(expected=ClassCastException.class) @@ -109,7 +112,7 @@ public void testExtractTimestampMillisException2() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); byte emptyBytes2[] = "".getBytes(); - jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes2)); + jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes2, timestamp)); } @Test diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java index 5b7e98748..314d07ee7 100644 --- a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -39,6 +39,7 @@ public class MessagePackParserTest extends TestCase { private Message mMessageWithMillisFloatTimestamp; private Message mMessageWithMillisStringTimestamp; private ObjectMapper mObjectMapper; + private long timestamp; @Override public void setUp() throws Exception { @@ -55,11 +56,12 @@ public void setUp() throws Exception { 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)); + mObjectMapper.writeValueAsBytes(mapWithSecondTimestamp), timestamp); HashMap mapWithMillisTimestamp = new HashMap(); mapWithMillisTimestamp.put("ts", 1405970352123l); @@ -67,7 +69,7 @@ public void setUp() throws Exception { mapWithMillisTimestamp.put("email", "alice@example.com"); mapWithMillisTimestamp.put("age", 27); mMessageWithMillisTimestamp = new Message("test", 0, 0, null, - mObjectMapper.writeValueAsBytes(mapWithMillisTimestamp)); + mObjectMapper.writeValueAsBytes(mapWithMillisTimestamp), timestamp); HashMap mapWithMillisFloatTimestamp = new HashMap(); @@ -76,7 +78,7 @@ public void setUp() throws Exception { mapWithMillisFloatTimestamp.put("email", "bob@example.com"); mapWithMillisFloatTimestamp.put("age", 35); mMessageWithMillisFloatTimestamp = new Message("test", 0, 0, null, - mObjectMapper.writeValueAsBytes(mapWithMillisFloatTimestamp)); + mObjectMapper.writeValueAsBytes(mapWithMillisFloatTimestamp), timestamp); HashMap mapWithMillisStringTimestamp = new HashMap(); mapWithMillisStringTimestamp.put("ts", "1405970352123"); @@ -84,10 +86,7 @@ public void setUp() throws Exception { mapWithMillisStringTimestamp.put("email", "charlie@example.com"); mapWithMillisStringTimestamp.put("age", 67); mMessageWithMillisStringTimestamp = new Message("test", 0, 0, null, - mObjectMapper.writeValueAsBytes(mapWithMillisStringTimestamp)); - - - + mObjectMapper.writeValueAsBytes(mapWithMillisStringTimestamp), timestamp); } @Test @@ -107,7 +106,7 @@ 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)); + mObjectMapper.writeValueAsBytes(mapWithoutTimestamp), timestamp); mMessagePackParser.extractTimestampMillis(nMessageWithoutTimestamp); } @@ -116,7 +115,7 @@ 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)); + mObjectMapper.writeValueAsBytes(mapWitUnsupportedFormatTimestamp), timestamp); mMessagePackParser.extractTimestampMillis(nMessageWithUnsupportedFormatTimestamp); } @@ -125,7 +124,7 @@ public void testNullTimestamp() throws Exception { HashMap mapWitNullTimestamp = new HashMap(); mapWitNullTimestamp.put("ts", null); Message nMessageWithNullTimestamp = new Message("test", 0, 0, null, - mObjectMapper.writeValueAsBytes(mapWitNullTimestamp)); + mObjectMapper.writeValueAsBytes(mapWitNullTimestamp), timestamp); mMessagePackParser.extractTimestampMillis(nMessageWithNullTimestamp); } diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index 7883a61ba..b6c650a4c 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -35,12 +35,13 @@ @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); + return new Message("test", 0, 0, null, data, timestamp); } @Override @@ -52,6 +53,8 @@ public void setUp() throws Exception { 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 @@ -73,11 +76,11 @@ public void testExtractPathTimestampMillis() throws Exception { UnitTestMessage1 message = UnitTestMessage1.newBuilder().setTimestamp(1405970352L).build(); assertEquals(1405970352000l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + 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()))); + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp))); } @Test @@ -92,11 +95,11 @@ public void testExtractNestedTimestampMillis() throws Exception { UnitTestMessage2 message = UnitTestMessage2.newBuilder() .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352L).build()).build(); assertEquals(1405970352000l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + 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()))); + 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 index c2c082861..5971b8d98 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java @@ -22,12 +22,13 @@ @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); + return new Message("test", 0, 0, null, data, timestamp); } @Override @@ -39,6 +40,8 @@ public void setUp() throws Exception { 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 @@ -65,13 +68,13 @@ public void testExtractPathTimestampMillis() throws Exception { TimestampedMessages.UnitTestTimestamp1 message = TimestampedMessages.UnitTestTimestamp1.newBuilder().setTimestamp(timestamp).build(); assertEquals(1405970352000l, - parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray()))); + 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()))); + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp.getSeconds()))); } @Test @@ -88,12 +91,12 @@ public void testExtractNestedTimestampMillis() throws Exception { 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()))); + 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()))); + 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 index 1c32af6cc..39e66b074 100644 --- a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -31,6 +31,7 @@ public class RegexMessageParserTest extends TestCase { private SecorConfig mConfig; private Message mMessageWithMillisTimestamp; private Message mMessageWithWrongFormatTimestamp; + private long timestamp; @Override public void setUp() throws Exception { @@ -44,13 +45,15 @@ public void setUp() throws Exception { 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); + 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); + mMessageWithWrongFormatTimestamp = new Message("test", 0, 0, null, messageWithWrongFormatTimestamp, timestamp); } @@ -65,7 +68,7 @@ public void testExtractTimestampMillis() throws Exception { public void testExtractTimestampMillisEmpty() throws Exception { RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); byte emptyBytes2[] = "".getBytes(); - regexMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes2)); + regexMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes2, timestamp)); } @Test(expected=NumberFormatException.class) diff --git a/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java index 95259a535..705979081 100644 --- a/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java @@ -37,6 +37,7 @@ public class SplitByFieldMessageParserTest extends TestCase { private Message mMessageWithTypeAndTimestamp; private Message mMessageWithoutTimestamp; private Message mMessageWithoutType; + private long timestamp; @Override public void setUp() throws Exception { @@ -53,17 +54,19 @@ public void setUp() throws Exception { 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); + 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); + 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); + mMessageWithoutType = new Message("test", 0, 0, null, messageWithoutType, timestamp); } @Test diff --git a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java index 543167910..c6ecc850b 100644 --- a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java @@ -15,6 +15,7 @@ @RunWith(PowerMockRunner.class) public class ThriftMessageParserTest extends TestCase { private SecorConfig mConfig; + private long timestamp; @Override public void setUp() throws Exception { @@ -25,6 +26,8 @@ public void setUp() throws Exception { 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 { @@ -32,7 +35,7 @@ private Message buildMessage(long timestamp, int timestampTwo, long timestampThr TSerializer serializer = new TSerializer(new TBinaryProtocol.Factory()); byte[] data = serializer.serialize(thriftMessage); - return new Message("test", 0, 0, null, data); + return new Message("test", 0, 0, null, data, timestamp); } @Test From 8fcd6a736405d5e3cbe540ac45f2d9643db928c2 Mon Sep 17 00:00:00 2001 From: tygrash Date: Wed, 17 May 2017 23:29:23 +0530 Subject: [PATCH 294/330] Added config property to use kafka message timestamp. --- src/main/config/secor.common.properties | 5 +++++ src/main/java/com/pinterest/secor/common/KafkaClient.java | 5 +++-- src/main/java/com/pinterest/secor/common/SecorConfig.java | 4 ++++ 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 857c2eadc..bddf14219 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -181,6 +181,10 @@ kafka.dual.commit.enabled=true # 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. +kafka.useKafkaTimestamp=false + # 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. @@ -404,3 +408,4 @@ parquet.validation=false 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/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index c6971fc58..c4d952d43 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -142,8 +142,9 @@ private Message getMessage(TopicPartition topicPartition, long offset, payload.get(payloadBytes); } Long timestamp = null; - timestamp = messageAndOffset.message().timestamp(); - + if (mConfig.getUseKafkaTimestamp()) { + timestamp = messageAndOffset.message().timestamp(); + } return new Message(topicPartition.getTopic(), topicPartition.getPartition(), messageAndOffset.offset(), keyBytes, payloadBytes, timestamp); } diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index f436b8c08..91cb5330e 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -131,6 +131,10 @@ public String getOffsetsStorage() { return getString("kafka.offsets.storage"); } + public boolean getUseKafkaTimestamp() { + return getBoolean("kafka.useKafkaTimestamp", false); + } + public int getGeneration() { return getInt("secor.generation"); } From a8b532f70915b43b378608d9533ff9849baa8a46 Mon Sep 17 00:00:00 2001 From: tygrash Date: Thu, 18 May 2017 01:07:31 +0530 Subject: [PATCH 295/330] Added usage of kafka timestamp in partition array generation --- .../secor/parser/DailyOffsetMessageParser.java | 8 +++----- .../secor/parser/JsonMessageParser.java | 17 ++++++++++------- .../secor/parser/MessagePackParser.java | 2 +- .../secor/parser/PartitionedMessageParser.java | 8 +++----- .../secor/parser/ProtobufMessageParser.java | 16 +++++++--------- .../secor/parser/RegexMessageParser.java | 8 +++++--- .../secor/parser/SplitByFieldMessageParser.java | 5 ++++- .../secor/parser/ThriftMessageParser.java | 10 +++++++--- .../secor/parser/TimestampedMessageParser.java | 4 +--- .../secor/parser/JsonMessageParserTest.java | 10 ++++++++++ .../secor/parser/MessagePackParserTest.java | 16 +++++++++++++++- .../secor/parser/ProtobufMessageParserTest.java | 9 +++++++++ .../secor/parser/RegexMessageParserTest.java | 8 ++++++++ .../parser/SplitByFieldMessageParserTest.java | 12 ++++++++++++ .../secor/parser/ThriftMessageParserTest.java | 15 ++++++++++++++- 15 files changed, 109 insertions(+), 39 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java index e653c6695..3f883a5c1 100644 --- a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java @@ -16,13 +16,12 @@ */ 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; -import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.message.Message; - /** @@ -49,7 +48,6 @@ public String[] extractPartitions(Message message) throws Exception { @Override public long extractTimestampMillis(final Message message) { - return new Date().getTime(); //Daily Timestamp generation + return (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : new Date().getTime(); //Daily Timestamp generation } - } diff --git a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java index eab1e3c57..54e63d718 100644 --- a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java @@ -36,15 +36,18 @@ public JsonMessageParser(SecorConfig config) { @Override public long extractTimestampMillis(final Message message) { JSONObject jsonObject = (JSONObject) JSONValue.parse(message.getPayload()); - if (jsonObject != null) { - Object fieldValue = getJsonFieldValue(jsonObject); - if (fieldValue != null) { - return toMillis(Double.valueOf(fieldValue.toString()).longValue()); + if (mConfig.getUseKafkaTimestamp()) { + return message.getTimestamp(); + } else { + if (jsonObject != null) { + Object fieldValue = getJsonFieldValue(jsonObject); + if (fieldValue != null) { + return toMillis(Double.valueOf(fieldValue.toString()).longValue()); + } + } else if (m_timestampRequired) { + throw new RuntimeException("Missing timestamp field for message: " + message); } - } 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 index 8baa4ffd4..67898d4ae 100644 --- a/src/main/java/com/pinterest/secor/parser/MessagePackParser.java +++ b/src/main/java/com/pinterest/secor/parser/MessagePackParser.java @@ -45,7 +45,7 @@ public MessagePackParser(SecorConfig config) { public long extractTimestampMillis(Message message) throws Exception { HashMap msgHash = mMessagePackObjectMapper.readValue(message.getPayload(), mTypeReference); - Object timestampValue = msgHash.get(mConfig.getMessageTimestampName()); + Object timestampValue = (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : msgHash.get(mConfig.getMessageTimestampName()); if (timestampValue instanceof Number) { return toMillis(((Number) timestampValue).longValue()); diff --git a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java index 508db3f1c..34f1fa82a 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java @@ -16,13 +16,12 @@ */ 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; -import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.message.Message; - /** @@ -46,7 +45,6 @@ public String[] extractPartitions(Message message) throws Exception { @Override public long extractTimestampMillis(final Message message) { - return new Date().getTime(); //Daily Timestamp generation + return (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : new Date().getTime(); //Daily Timestamp generation } - } diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index 3ae8b9a16..dabbf8c7f 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -16,18 +16,16 @@ */ 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.google.protobuf.util.Timestamps; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import com.pinterest.secor.util.ProtobufUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.regex.Pattern; /** * Protocol buffer message timestamp extractor @@ -67,7 +65,7 @@ public ProtobufMessageParser(SecorConfig config) { @Override public long extractTimestampMillis(final Message message) throws IOException { - return extractTimestampMillis(message.getTopic(), message.getPayload()); + return (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : extractTimestampMillis(message.getTopic(), message.getPayload()); } public long extractTimestampMillis(String topic, final byte[] bytes) throws IOException { diff --git a/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java index 11a8a41cc..2d19b8485 100644 --- a/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java @@ -18,11 +18,11 @@ 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; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * RegexMessageParser extracts timestamp field (specified by 'message.timestamp.input.pattern') @@ -44,6 +44,9 @@ public RegexMessageParser(SecorConfig config) { @Override public long extractTimestampMillis(final Message message) { + if (mConfig.getUseKafkaTimestamp()) { + return message.getTimestamp(); + } String line = new String(message.getPayload()); Matcher m = mTsPattern.matcher(line); if (m.find()) { @@ -54,5 +57,4 @@ public long extractTimestampMillis(final Message message) { } 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 index 9b76f84d2..72e748d8b 100644 --- a/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java @@ -45,6 +45,9 @@ public SplitByFieldMessageParser(SecorConfig config) { @Override public long extractTimestampMillis(Message message) throws Exception { + if (mConfig.getUseKafkaTimestamp()) { + return message.getTimestamp(); + } throw new UnsupportedOperationException("Unsupported, use extractPartitions method instead"); } @@ -56,7 +59,7 @@ public String[] extractPartitions(Message message) throws Exception { } String eventType = extractEventType(jsonObject); - long timestampMillis = extractTimestampMillis(jsonObject); + long timestampMillis = (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : extractTimestampMillis(jsonObject); String[] timestampPartitions = generatePartitions(timestampMillis, mUsingHourly, mUsingMinutely); return ArrayUtils.addAll(new String[]{eventType}, timestampPartitions); diff --git a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java index f3349c9b1..6cfa2e6a4 100644 --- a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java @@ -77,10 +77,14 @@ public ThriftMessageParser(SecorConfig config) @Override public long extractTimestampMillis(final Message message) throws TException { long timestamp; - if ("i32".equals(mTimestampType)) { - timestamp = (long) mDeserializer.partialDeserializeI32(message.getPayload(), mThriftPath); + if (mConfig.getUseKafkaTimestamp()) { + timestamp = message.getTimestamp(); } else { - timestamp = mDeserializer.partialDeserializeI64(message.getPayload(), mThriftPath); + if ("i32".equals(mTimestampType)) { + timestamp = (long) mDeserializer.partialDeserializeI32(message.getPayload(), mThriftPath); + } else { + timestamp = mDeserializer.partialDeserializeI64(message.getPayload(), mThriftPath); + } } 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 03910764c..6c60a6b7d 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -16,8 +16,6 @@ */ package com.pinterest.secor.parser; -import com.google.protobuf.Timestamp; -import com.google.protobuf.util.Timestamps; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import org.slf4j.Logger; @@ -269,4 +267,4 @@ public String[] getPreviousPartitions(String[] partitions) throws Exception { } return generatePartitions(millis, usingHourly, usingMinutely); } - } +} diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index e2e5d8419..6c30f3407 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -77,6 +77,16 @@ public void setUp() throws Exception { mMessageWithNestedTimestamp = new Message("test", 0, 0, null, messageWithNestedTimestamp, timestamp); } + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); + JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); + + assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithSecondsTimestamp)); + assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithMillisTimestamp)); + assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithMillisFloatTimestamp)); + } + @Test public void testExtractTimestampMillis() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java index 314d07ee7..717836f10 100644 --- a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -33,6 +33,7 @@ @RunWith(PowerMockRunner.class) public class MessagePackParserTest extends TestCase { + private SecorConfig mConfig; private MessagePackParser mMessagePackParser; private Message mMessageWithSecondsTimestamp; private Message mMessageWithMillisTimestamp; @@ -43,7 +44,7 @@ public class MessagePackParserTest extends TestCase { @Override public void setUp() throws Exception { - SecorConfig mConfig = Mockito.mock(SecorConfig.class); + 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"); @@ -89,6 +90,19 @@ public void setUp() throws Exception { mObjectMapper.writeValueAsBytes(mapWithMillisStringTimestamp), timestamp); } + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); + assertEquals(timestamp, mMessagePackParser.extractTimestampMillis( + mMessageWithSecondsTimestamp)); + assertEquals(timestamp, mMessagePackParser.extractTimestampMillis( + mMessageWithMillisTimestamp)); + assertEquals(timestamp, mMessagePackParser.extractTimestampMillis( + mMessageWithMillisFloatTimestamp)); + assertEquals(timestamp, mMessagePackParser.extractTimestampMillis( + mMessageWithMillisStringTimestamp)); + } + @Test public void testExtractTimestampMillis() throws Exception { assertEquals(1405970352000l, mMessagePackParser.extractTimestampMillis( diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index b6c650a4c..ea19c0ea0 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -57,6 +57,15 @@ public void setUp() throws Exception { timestamp = System.currentTimeMillis(); } + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + assertEquals(1405970352l, parser.extractTimestampMillis(buildMessage(1405970352l))); + assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); + } + @Test public void testExtractTimestampMillis() throws Exception { ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java index 39e66b074..2ddd089a1 100644 --- a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -57,6 +57,14 @@ public void setUp() throws Exception { } + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); + RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); + + assertEquals(timestamp, regexMessageParser.extractTimestampMillis(mMessageWithMillisTimestamp)); + } + @Test public void testExtractTimestampMillis() throws Exception { RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java index 705979081..107058eaf 100644 --- a/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java @@ -69,6 +69,18 @@ public void setUp() throws Exception { mMessageWithoutType = new Message("test", 0, 0, null, messageWithoutType, timestamp); } + @Test + public void testExtractTypeAndTimestampFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithTypeAndTimestamp)); + assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithoutType)); + + assertEquals("event1", jsonMessageParser.extractEventType((JSONObject) JSONValue.parse(mMessageWithTypeAndTimestamp.getPayload()))); + assertEquals("event2", jsonMessageParser.extractEventType((JSONObject) JSONValue.parse(mMessageWithoutTimestamp.getPayload()))); + } + @Test public void testExtractTypeAndTimestamp() throws Exception { SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java index c6ecc850b..a8ca4c8aa 100644 --- a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java @@ -38,6 +38,20 @@ private Message buildMessage(long timestamp, int timestampTwo, long timestampThr return new Message("test", 0, 0, null, data, timestamp); } + @Test + public void testExtractTimestampFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getUseKafkaTimestamp()).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"); @@ -87,5 +101,4 @@ public void testAttemptExtractInvalidField() throws Exception { parser.extractTimestampMillis(buildMessage(1L, 2, 3L)); } - } From 4e2d9e0c97b93c504ed6c62f7133e2099cc127c6 Mon Sep 17 00:00:00 2001 From: tygrash Date: Thu, 18 May 2017 10:16:23 +0530 Subject: [PATCH 296/330] Revert "Added usage of kafka timestamp in partition array generation" This reverts commit a8b532f70915b43b378608d9533ff9849baa8a46. --- .../secor/parser/DailyOffsetMessageParser.java | 8 +++++--- .../secor/parser/JsonMessageParser.java | 17 +++++++---------- .../secor/parser/MessagePackParser.java | 2 +- .../secor/parser/PartitionedMessageParser.java | 8 +++++--- .../secor/parser/ProtobufMessageParser.java | 16 +++++++++------- .../secor/parser/RegexMessageParser.java | 8 +++----- .../secor/parser/SplitByFieldMessageParser.java | 5 +---- .../secor/parser/ThriftMessageParser.java | 10 +++------- .../secor/parser/TimestampedMessageParser.java | 4 +++- .../secor/parser/JsonMessageParserTest.java | 10 ---------- .../secor/parser/MessagePackParserTest.java | 16 +--------------- .../secor/parser/ProtobufMessageParserTest.java | 9 --------- .../secor/parser/RegexMessageParserTest.java | 8 -------- .../parser/SplitByFieldMessageParserTest.java | 12 ------------ .../secor/parser/ThriftMessageParserTest.java | 15 +-------------- 15 files changed, 39 insertions(+), 109 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java index 3f883a5c1..e653c6695 100644 --- a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java @@ -16,12 +16,13 @@ */ 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; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; + /** @@ -48,6 +49,7 @@ public String[] extractPartitions(Message message) throws Exception { @Override public long extractTimestampMillis(final Message message) { - return (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : new Date().getTime(); //Daily Timestamp generation + return new Date().getTime(); //Daily Timestamp generation } + } diff --git a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java index 54e63d718..eab1e3c57 100644 --- a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java @@ -36,18 +36,15 @@ public JsonMessageParser(SecorConfig config) { @Override public long extractTimestampMillis(final Message message) { JSONObject jsonObject = (JSONObject) JSONValue.parse(message.getPayload()); - if (mConfig.getUseKafkaTimestamp()) { - return message.getTimestamp(); - } else { - if (jsonObject != null) { - Object fieldValue = getJsonFieldValue(jsonObject); - if (fieldValue != null) { - return toMillis(Double.valueOf(fieldValue.toString()).longValue()); - } - } else if (m_timestampRequired) { - throw new RuntimeException("Missing timestamp field for message: " + message); + if (jsonObject != null) { + Object fieldValue = getJsonFieldValue(jsonObject); + if (fieldValue != null) { + 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 index 67898d4ae..8baa4ffd4 100644 --- a/src/main/java/com/pinterest/secor/parser/MessagePackParser.java +++ b/src/main/java/com/pinterest/secor/parser/MessagePackParser.java @@ -45,7 +45,7 @@ public MessagePackParser(SecorConfig config) { public long extractTimestampMillis(Message message) throws Exception { HashMap msgHash = mMessagePackObjectMapper.readValue(message.getPayload(), mTypeReference); - Object timestampValue = (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : msgHash.get(mConfig.getMessageTimestampName()); + Object timestampValue = msgHash.get(mConfig.getMessageTimestampName()); if (timestampValue instanceof Number) { return toMillis(((Number) timestampValue).longValue()); diff --git a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java index 34f1fa82a..508db3f1c 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java @@ -16,12 +16,13 @@ */ 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; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; + /** @@ -45,6 +46,7 @@ public String[] extractPartitions(Message message) throws Exception { @Override public long extractTimestampMillis(final Message message) { - return (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : new Date().getTime(); //Daily Timestamp generation + return new Date().getTime(); //Daily Timestamp generation } + } diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java index dabbf8c7f..3ae8b9a16 100644 --- a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -16,16 +16,18 @@ */ package com.pinterest.secor.parser; -import com.google.protobuf.CodedInputStream; +import java.io.IOException; +import java.util.regex.Pattern; + +import com.google.protobuf.Descriptors; import com.google.protobuf.util.Timestamps; -import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.message.Message; -import com.pinterest.secor.util.ProtobufUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.regex.Pattern; +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 @@ -65,7 +67,7 @@ public ProtobufMessageParser(SecorConfig config) { @Override public long extractTimestampMillis(final Message message) throws IOException { - return (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : extractTimestampMillis(message.getTopic(), message.getPayload()); + return extractTimestampMillis(message.getTopic(), message.getPayload()); } public long extractTimestampMillis(String topic, final byte[] bytes) throws IOException { diff --git a/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java index 2d19b8485..11a8a41cc 100644 --- a/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java @@ -18,11 +18,11 @@ 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * RegexMessageParser extracts timestamp field (specified by 'message.timestamp.input.pattern') @@ -44,9 +44,6 @@ public RegexMessageParser(SecorConfig config) { @Override public long extractTimestampMillis(final Message message) { - if (mConfig.getUseKafkaTimestamp()) { - return message.getTimestamp(); - } String line = new String(message.getPayload()); Matcher m = mTsPattern.matcher(line); if (m.find()) { @@ -57,4 +54,5 @@ public long extractTimestampMillis(final Message message) { } 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 index 72e748d8b..9b76f84d2 100644 --- a/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java @@ -45,9 +45,6 @@ public SplitByFieldMessageParser(SecorConfig config) { @Override public long extractTimestampMillis(Message message) throws Exception { - if (mConfig.getUseKafkaTimestamp()) { - return message.getTimestamp(); - } throw new UnsupportedOperationException("Unsupported, use extractPartitions method instead"); } @@ -59,7 +56,7 @@ public String[] extractPartitions(Message message) throws Exception { } String eventType = extractEventType(jsonObject); - long timestampMillis = (mConfig.getUseKafkaTimestamp()) ? message.getTimestamp() : extractTimestampMillis(jsonObject); + long timestampMillis = extractTimestampMillis(jsonObject); String[] timestampPartitions = generatePartitions(timestampMillis, mUsingHourly, mUsingMinutely); return ArrayUtils.addAll(new String[]{eventType}, timestampPartitions); diff --git a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java index 6cfa2e6a4..f3349c9b1 100644 --- a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java @@ -77,14 +77,10 @@ public ThriftMessageParser(SecorConfig config) @Override public long extractTimestampMillis(final Message message) throws TException { long timestamp; - if (mConfig.getUseKafkaTimestamp()) { - timestamp = message.getTimestamp(); + if ("i32".equals(mTimestampType)) { + timestamp = (long) mDeserializer.partialDeserializeI32(message.getPayload(), mThriftPath); } else { - if ("i32".equals(mTimestampType)) { - timestamp = (long) mDeserializer.partialDeserializeI32(message.getPayload(), mThriftPath); - } else { - timestamp = mDeserializer.partialDeserializeI64(message.getPayload(), mThriftPath); - } + timestamp = mDeserializer.partialDeserializeI64(message.getPayload(), mThriftPath); } 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 6c60a6b7d..03910764c 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -16,6 +16,8 @@ */ package com.pinterest.secor.parser; +import com.google.protobuf.Timestamp; +import com.google.protobuf.util.Timestamps; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import org.slf4j.Logger; @@ -267,4 +269,4 @@ public String[] getPreviousPartitions(String[] partitions) throws Exception { } return generatePartitions(millis, usingHourly, usingMinutely); } -} + } diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index 6c30f3407..e2e5d8419 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -77,16 +77,6 @@ public void setUp() throws Exception { mMessageWithNestedTimestamp = new Message("test", 0, 0, null, messageWithNestedTimestamp, timestamp); } - @Test - public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { - Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); - JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); - - assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithSecondsTimestamp)); - assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithMillisTimestamp)); - assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithMillisFloatTimestamp)); - } - @Test public void testExtractTimestampMillis() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java index 717836f10..314d07ee7 100644 --- a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -33,7 +33,6 @@ @RunWith(PowerMockRunner.class) public class MessagePackParserTest extends TestCase { - private SecorConfig mConfig; private MessagePackParser mMessagePackParser; private Message mMessageWithSecondsTimestamp; private Message mMessageWithMillisTimestamp; @@ -44,7 +43,7 @@ public class MessagePackParserTest extends TestCase { @Override public void setUp() throws Exception { - mConfig = Mockito.mock(SecorConfig.class); + SecorConfig 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"); @@ -90,19 +89,6 @@ public void setUp() throws Exception { mObjectMapper.writeValueAsBytes(mapWithMillisStringTimestamp), timestamp); } - @Test - public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { - Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); - assertEquals(timestamp, mMessagePackParser.extractTimestampMillis( - mMessageWithSecondsTimestamp)); - assertEquals(timestamp, mMessagePackParser.extractTimestampMillis( - mMessageWithMillisTimestamp)); - assertEquals(timestamp, mMessagePackParser.extractTimestampMillis( - mMessageWithMillisFloatTimestamp)); - assertEquals(timestamp, mMessagePackParser.extractTimestampMillis( - mMessageWithMillisStringTimestamp)); - } - @Test public void testExtractTimestampMillis() throws Exception { assertEquals(1405970352000l, mMessagePackParser.extractTimestampMillis( diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index ea19c0ea0..b6c650a4c 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -57,15 +57,6 @@ public void setUp() throws Exception { timestamp = System.currentTimeMillis(); } - @Test - public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { - Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); - ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); - - assertEquals(1405970352l, parser.extractTimestampMillis(buildMessage(1405970352l))); - assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); - } - @Test public void testExtractTimestampMillis() throws Exception { ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java index 2ddd089a1..39e66b074 100644 --- a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -57,14 +57,6 @@ public void setUp() throws Exception { } - @Test - public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { - Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); - RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); - - assertEquals(timestamp, regexMessageParser.extractTimestampMillis(mMessageWithMillisTimestamp)); - } - @Test public void testExtractTimestampMillis() throws Exception { RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java index 107058eaf..705979081 100644 --- a/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java @@ -69,18 +69,6 @@ public void setUp() throws Exception { mMessageWithoutType = new Message("test", 0, 0, null, messageWithoutType, timestamp); } - @Test - public void testExtractTypeAndTimestampFromKafkaTimestamp() throws Exception { - Mockito.when(mConfig.getUseKafkaTimestamp()).thenReturn(true); - SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); - - assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithTypeAndTimestamp)); - assertEquals(timestamp, jsonMessageParser.extractTimestampMillis(mMessageWithoutType)); - - assertEquals("event1", jsonMessageParser.extractEventType((JSONObject) JSONValue.parse(mMessageWithTypeAndTimestamp.getPayload()))); - assertEquals("event2", jsonMessageParser.extractEventType((JSONObject) JSONValue.parse(mMessageWithoutTimestamp.getPayload()))); - } - @Test public void testExtractTypeAndTimestamp() throws Exception { SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java index a8ca4c8aa..c6ecc850b 100644 --- a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java @@ -38,20 +38,6 @@ private Message buildMessage(long timestamp, int timestampTwo, long timestampThr return new Message("test", 0, 0, null, data, timestamp); } - @Test - public void testExtractTimestampFromKafkaTimestamp() throws Exception { - Mockito.when(mConfig.getUseKafkaTimestamp()).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"); @@ -101,4 +87,5 @@ public void testAttemptExtractInvalidField() throws Exception { parser.extractTimestampMillis(buildMessage(1L, 2, 3L)); } + } From cff776388fc497d4193d910feb182eeffeab74ab Mon Sep 17 00:00:00 2001 From: tygrash Date: Thu, 18 May 2017 13:32:09 +0530 Subject: [PATCH 297/330] Adding kafka timestamp extraction logic in base parser class. Added tests for inherited classes. --- src/main/config/secor.common.properties | 2 +- .../pinterest/secor/common/SecorConfig.java | 2 +- .../parser/TimestampedMessageParser.java | 24 +++++++++----- .../secor/tools/ProgressMonitor.java | 2 +- .../secor/parser/JsonMessageParserTest.java | 29 +++++++++++------ .../secor/parser/MessagePackParserTest.java | 32 ++++++++++++++----- .../parser/ProtobufMessageParserTest.java | 9 ++++++ .../parser/ProtobufTimestampParserTest.java | 9 ++++++ .../secor/parser/RegexMessageParserTest.java | 8 +++++ .../secor/parser/ThriftMessageParserTest.java | 15 ++++++++- 10 files changed, 103 insertions(+), 29 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index bddf14219..2c0ce5d20 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -183,7 +183,7 @@ 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. -kafka.useKafkaTimestamp=false +kafka.useTimestamp=false # 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. diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 91cb5330e..a38afed76 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -132,7 +132,7 @@ public String getOffsetsStorage() { } public boolean getUseKafkaTimestamp() { - return getBoolean("kafka.useKafkaTimestamp", false); + return getBoolean("kafka.useTimestamp", false); } public int getGeneration() { diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 03910764c..65b8843b9 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -16,8 +16,6 @@ */ package com.pinterest.secor.parser; -import com.google.protobuf.Timestamp; -import com.google.protobuf.util.Timestamps; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import org.slf4j.Logger; @@ -58,12 +56,15 @@ public abstract class TimestampedMessageParser extends MessageParser implements protected final boolean mUsingHourly; protected final boolean mUsingMinutely; + protected final boolean mUseKafkaTimestamp; + public TimestampedMessageParser(SecorConfig config) { super(config); mUsingHourly = usingHourly(config); mUsingMinutely = usingMinutely(config); + mUseKafkaTimestamp = useKafkaTimestamp(config); mDtFormat = usingDateFormat(config); mHrFormat = usingHourFormat(config); mMinFormat = usingMinuteFormat(config); @@ -91,7 +92,6 @@ public TimestampedMessageParser(SecorConfig config) { mDtHrMinFormatter = new SimpleDateFormat(mDtFormat+ "-" + mHrFormat + "-" + mMinFormat); mDtHrMinFormatter.setTimeZone(config.getTimeZone()); - } static boolean usingHourly(SecorConfig config) { @@ -126,6 +126,10 @@ static String usingMinutePrefix(SecorConfig config) { return config.getString("partitioner.granularity.minute.prefix", "min="); } + static boolean useKafkaTimestamp(SecorConfig config) { + return config.getBoolean("kafka.useTimestamp", false); + } + 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); @@ -145,6 +149,10 @@ protected static long toMillis(final long timestamp) { 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); @@ -172,14 +180,14 @@ protected long parsePartitions(String[] partitions) throws Exception { @Override public String[] extractPartitions(Message message) throws Exception { // Date constructor takes milliseconds since epoch. - long timestampMillis = extractTimestampMillis(message); + long timestampMillis = getTimestampMillis(message); return generatePartitions(timestampMillis, mUsingHourly, mUsingMinutely); } private long getFinalizedTimestampMillis(Message lastMessage, Message committedMessage) throws Exception { - long lastTimestamp = extractTimestampMillis(lastMessage); - long committedTimestamp = extractTimestampMillis(committedMessage); + long lastTimestamp = getTimestampMillis(lastMessage); + long committedTimestamp = getTimestampMillis(committedMessage); long now = System.currentTimeMillis(); if (lastTimestamp == committedTimestamp && (now - lastTimestamp) > mFinalizerDelaySeconds * 1000) { @@ -216,8 +224,8 @@ public String[] getFinalizedUptoPartitions(List lastMessages, 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); @@ -269,4 +277,4 @@ public String[] getPreviousPartitions(String[] partitions) throws Exception { } return generatePartitions(millis, usingHourly, usingMinutely); } - } +} diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index f9f06c140..aef5b0098 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -235,7 +235,7 @@ private String metricName(String key) { private long getTimestamp(Message message) throws Exception { if (mMessageParser instanceof TimestampedMessageParser) { - return ((TimestampedMessageParser)mMessageParser).extractTimestampMillis(message); + return ((TimestampedMessageParser)mMessageParser).getTimestampMillis(message); } else { return -1; } diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index e2e5d8419..6db551a51 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -70,24 +70,34 @@ public void setUp() throws Exception { 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, null, messageWithoutTimestamp, timestamp); + mMessageWithoutTimestamp = new Message("test", 0, 0, null, messageWithoutTimestamp, null); 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.getBoolean("kafka.useTimestamp", false)).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(1405911096000l, jsonMessageParser.extractTimestampMillis(mMessageWithSecondsTimestamp)); - assertEquals(1405911096123l, jsonMessageParser.extractTimestampMillis(mMessageWithMillisTimestamp)); - assertEquals(1405911096123l, 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 @@ -96,7 +106,7 @@ public void testExtractNestedTimestampMillis() throws Exception { Mockito.when(mConfig.getMessageTimestampName()).thenReturn("meta_data.created"); JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); - assertEquals(1405911096123l, jsonMessageParser.extractTimestampMillis(mMessageWithNestedTimestamp)); + assertEquals(1405911096123l, jsonMessageParser.getTimestampMillis(mMessageWithNestedTimestamp)); } @Test(expected=ClassCastException.class) @@ -104,7 +114,7 @@ public void testExtractTimestampMillisException1() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); byte emptyBytes1[] = {}; - jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes1, timestamp)); + jsonMessageParser.getTimestampMillis(new Message("test", 0, 0, null, emptyBytes1, timestamp)); } @Test(expected=ClassCastException.class) @@ -112,7 +122,7 @@ public void testExtractTimestampMillisException2() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); byte emptyBytes2[] = "".getBytes(); - jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes2, timestamp)); + jsonMessageParser.getTimestampMillis(new Message("test", 0, 0, null, emptyBytes2, timestamp)); } @Test @@ -265,7 +275,8 @@ public void testHourlyGetFinalizedUptoPartitions() throws Exception { assertEquals(expectedPartition, retrievedPartition); } } -@Test + + @Test public void testMinutelyGetFinalizedUptoPartitions() throws Exception { Mockito.when(TimestampedMessageParser.usingMinutely(mConfig)).thenReturn(true); JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java index 314d07ee7..27225de09 100644 --- a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -33,6 +33,7 @@ @RunWith(PowerMockRunner.class) public class MessagePackParserTest extends TestCase { + SecorConfig mConfig; private MessagePackParser mMessagePackParser; private Message mMessageWithSecondsTimestamp; private Message mMessageWithMillisTimestamp; @@ -43,7 +44,7 @@ public class MessagePackParserTest extends TestCase { @Override public void setUp() throws Exception { - SecorConfig mConfig = Mockito.mock(SecorConfig.class); + 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"); @@ -89,15 +90,30 @@ public void setUp() throws Exception { mObjectMapper.writeValueAsBytes(mapWithMillisStringTimestamp), timestamp); } + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getBoolean("kafka.useTimestamp", false)).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.extractTimestampMillis( + assertEquals(1405970352000l, mMessagePackParser.getTimestampMillis( mMessageWithSecondsTimestamp)); - assertEquals(1405970352123l, mMessagePackParser.extractTimestampMillis( + assertEquals(1405970352123l, mMessagePackParser.getTimestampMillis( mMessageWithMillisTimestamp)); - assertEquals(1405970352123l, mMessagePackParser.extractTimestampMillis( + assertEquals(1405970352123l, mMessagePackParser.getTimestampMillis( mMessageWithMillisFloatTimestamp)); - assertEquals(1405970352123l, mMessagePackParser.extractTimestampMillis( + assertEquals(1405970352123l, mMessagePackParser.getTimestampMillis( mMessageWithMillisStringTimestamp)); } @@ -107,7 +123,7 @@ public void testMissingTimestamp() throws Exception { mapWithoutTimestamp.put("email", "mary@example.com"); Message nMessageWithoutTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWithoutTimestamp), timestamp); - mMessagePackParser.extractTimestampMillis(nMessageWithoutTimestamp); + mMessagePackParser.getTimestampMillis(nMessageWithoutTimestamp); } @Test(expected=NumberFormatException.class) @@ -116,7 +132,7 @@ public void testUnsupportedTimestampFormat() throws Exception { mapWitUnsupportedFormatTimestamp.put("ts", "2014-11-14T18:12:52.878Z"); Message nMessageWithUnsupportedFormatTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWitUnsupportedFormatTimestamp), timestamp); - mMessagePackParser.extractTimestampMillis(nMessageWithUnsupportedFormatTimestamp); + mMessagePackParser.getTimestampMillis(nMessageWithUnsupportedFormatTimestamp); } @Test(expected=NullPointerException.class) @@ -125,7 +141,7 @@ public void testNullTimestamp() throws Exception { mapWitNullTimestamp.put("ts", null); Message nMessageWithNullTimestamp = new Message("test", 0, 0, null, mObjectMapper.writeValueAsBytes(mapWitNullTimestamp), timestamp); - mMessagePackParser.extractTimestampMillis(nMessageWithNullTimestamp); + mMessagePackParser.getTimestampMillis(nMessageWithNullTimestamp); } @Test diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java index b6c650a4c..ce92dc147 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java @@ -57,6 +57,15 @@ public void setUp() throws Exception { 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); diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java index 5971b8d98..1c662165c 100644 --- a/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java @@ -44,6 +44,15 @@ public void setUp() throws Exception { 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); diff --git a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java index 39e66b074..2ad851170 100644 --- a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -57,6 +57,14 @@ public void setUp() throws Exception { } + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getBoolean("kafka.useTimestamp", false)).thenReturn(true); + RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); + + assertEquals(timestamp, regexMessageParser.getTimestampMillis(mMessageWithMillisTimestamp)); + } + @Test public void testExtractTimestampMillis() throws Exception { RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); diff --git a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java index c6ecc850b..32692d7ce 100644 --- a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java @@ -38,6 +38,20 @@ private Message buildMessage(long timestamp, int timestampTwo, long timestampThr 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"); @@ -87,5 +101,4 @@ public void testAttemptExtractInvalidField() throws Exception { parser.extractTimestampMillis(buildMessage(1L, 2, 3L)); } - } From fbeb3aff953b87b793f5801444dd64215e774fe3 Mon Sep 17 00:00:00 2001 From: tygrash Date: Fri, 19 May 2017 00:19:48 +0530 Subject: [PATCH 298/330] Calling config.useKafkaTimestamp() instead of extracting it again --- src/main/java/com/pinterest/secor/common/KafkaClient.java | 2 +- src/main/java/com/pinterest/secor/common/SecorConfig.java | 2 +- src/main/java/com/pinterest/secor/parser/DateMessageParser.java | 1 - .../com/pinterest/secor/parser/TimestampedMessageParser.java | 2 +- .../java/com/pinterest/secor/parser/JsonMessageParserTest.java | 2 +- .../java/com/pinterest/secor/parser/MessagePackParserTest.java | 2 +- .../java/com/pinterest/secor/parser/RegexMessageParserTest.java | 2 +- 7 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index c4d952d43..8a0add479 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -142,7 +142,7 @@ private Message getMessage(TopicPartition topicPartition, long offset, payload.get(payloadBytes); } Long timestamp = null; - if (mConfig.getUseKafkaTimestamp()) { + if (mConfig.useKafkaTimestamp()) { timestamp = messageAndOffset.message().timestamp(); } return new Message(topicPartition.getTopic(), topicPartition.getPartition(), diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index a38afed76..e0d067762 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -131,7 +131,7 @@ public String getOffsetsStorage() { return getString("kafka.offsets.storage"); } - public boolean getUseKafkaTimestamp() { + public boolean useKafkaTimestamp() { return getBoolean("kafka.useTimestamp", false); } diff --git a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java index e47f47406..0eb3606da 100644 --- a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java @@ -82,5 +82,4 @@ public String[] extractPartitions(Message message) { return result; } - } diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 65b8843b9..ec6593ae3 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -127,7 +127,7 @@ static String usingMinutePrefix(SecorConfig config) { } static boolean useKafkaTimestamp(SecorConfig config) { - return config.getBoolean("kafka.useTimestamp", false); + return config.useKafkaTimestamp(); } protected static long toMillis(final long timestamp) { diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index 6db551a51..01b3af3eb 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -79,7 +79,7 @@ public void setUp() throws Exception { @Test public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { - Mockito.when(mConfig.getBoolean("kafka.useTimestamp", false)).thenReturn(true); + Mockito.when(mConfig.useKafkaTimestamp()).thenReturn(true); JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); assertEquals(timestamp, jsonMessageParser.getTimestampMillis(mMessageWithSecondsTimestamp)); diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java index 27225de09..70c6850c7 100644 --- a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -92,7 +92,7 @@ public void setUp() throws Exception { @Test public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { - Mockito.when(mConfig.getBoolean("kafka.useTimestamp", false)).thenReturn(true); + Mockito.when(mConfig.useKafkaTimestamp()).thenReturn(true); mMessagePackParser = new MessagePackParser(mConfig); assertEquals(timestamp, mMessagePackParser.getTimestampMillis( diff --git a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java index 2ad851170..5f9bcbdab 100644 --- a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -59,7 +59,7 @@ public void setUp() throws Exception { @Test public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { - Mockito.when(mConfig.getBoolean("kafka.useTimestamp", false)).thenReturn(true); + Mockito.when(mConfig.useKafkaTimestamp()).thenReturn(true); RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); assertEquals(timestamp, regexMessageParser.getTimestampMillis(mMessageWithMillisTimestamp)); From 40773feb41c72232e4d43f73bacd04c7b21aa2d0 Mon Sep 17 00:00:00 2001 From: tygrash Date: Sat, 20 May 2017 00:25:02 +0530 Subject: [PATCH 299/330] Calling check-safe extraction for metrics class --- src/main/java/com/pinterest/secor/common/SecorConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index e0d067762..c1c3526a7 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -571,7 +571,7 @@ public String getThriftProtocolClass() { } public String getMetricsCollectorClass() { - return mProperties.getString("secor.monitoring.metrics.collector.class"); + return getString("secor.monitoring.metrics.collector.class"); } /** From f1913ccf5f130575e038fe5e1a81fd90074698a4 Mon Sep 17 00:00:00 2001 From: tygrash Date: Sat, 20 May 2017 00:48:14 +0530 Subject: [PATCH 300/330] Removing unwanted new lines --- .../secor/parser/DailyOffsetMessageParser.java | 7 ++----- .../secor/parser/Iso8601MessageParser.java | 13 +++++-------- .../pinterest/secor/parser/JsonMessageParser.java | 1 - .../pinterest/secor/parser/PartitionFinalizer.java | 1 - .../secor/parser/PartitionedMessageParser.java | 8 ++------ .../pinterest/secor/parser/RegexMessageParser.java | 5 ++--- 6 files changed, 11 insertions(+), 24 deletions(-) diff --git a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java index e653c6695..1ea49b568 100644 --- a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java @@ -16,14 +16,11 @@ */ package com.pinterest.secor.parser; -import org.apache.commons.lang.StringUtils; - -import java.util.Date; - 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. diff --git a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java index 7b591eaf4..6b18215d2 100644 --- a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java @@ -16,18 +16,16 @@ */ package com.pinterest.secor.parser; -import java.text.SimpleDateFormat; -import java.util.Date; - +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; import net.minidev.json.JSONObject; import net.minidev.json.JSONValue; -import javax.xml.bind.DatatypeConverter; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.message.Message; +import javax.xml.bind.DatatypeConverter; +import java.text.SimpleDateFormat; +import java.util.Date; /** * Iso8601MessageParser extracts timestamp field (specified by 'message.timestamp.name') @@ -68,5 +66,4 @@ public String[] extractPartitions(Message message) throws Exception { return result; } - } diff --git a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java index eab1e3c57..0d09b7e34 100644 --- a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java @@ -46,5 +46,4 @@ public long extractTimestampMillis(final Message message) { } return 0; } - } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java index bac754d78..3ab672895 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -192,7 +192,6 @@ private void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throw LOG.info("touching file {}", successFilePath); FileUtil.touch(successFilePath); } - } public void finalizePartitions() throws Exception { diff --git a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java index 508db3f1c..335203caf 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java @@ -16,14 +16,11 @@ */ package com.pinterest.secor.parser; -import org.apache.commons.lang.StringUtils; - -import java.util.Date; - 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. @@ -48,5 +45,4 @@ public String[] extractPartitions(Message message) throws Exception { public long extractTimestampMillis(final Message message) { return new Date().getTime(); //Daily Timestamp generation } - } diff --git a/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java index 11a8a41cc..04fc482ee 100644 --- a/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java @@ -18,11 +18,11 @@ 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; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * RegexMessageParser extracts timestamp field (specified by 'message.timestamp.input.pattern') @@ -54,5 +54,4 @@ public long extractTimestampMillis(final Message message) { } throw new NumberFormatException("Cannot find timestamp field in: " + line); } - } From f14d109e277d3a79e163fd025707dbceaa4e2503 Mon Sep 17 00:00:00 2001 From: Nacho Munoz Date: Mon, 12 Jun 2017 15:49:33 +0100 Subject: [PATCH 301/330] Stores Kafka message timestamp as part of the encoded information --- .../java/com/pinterest/secor/io/KeyValue.java | 28 +++++++++++-- ...gePackSequenceFileReaderWriterFactory.java | 42 +++++++++++++++---- .../pinterest/secor/writer/MessageWriter.java | 2 +- ...ckSequenceFileReaderWriterFactoryTest.java | 18 +++++++- 4 files changed, 77 insertions(+), 13 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/KeyValue.java b/src/main/java/com/pinterest/secor/io/KeyValue.java index c7f5721a9..6dd60f3bd 100644 --- a/src/main/java/com/pinterest/secor/io/KeyValue.java +++ b/src/main/java/com/pinterest/secor/io/KeyValue.java @@ -19,21 +19,23 @@ /** * 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 @@ -41,6 +43,15 @@ 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() { @@ -50,9 +61,20 @@ public long getOffset() { 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/MessagePackSequenceFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactory.java index 187f0f207..6a08eb421 100644 --- a/src/main/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactory.java +++ b/src/main/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactory.java @@ -44,6 +44,7 @@ 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 @@ -76,6 +77,7 @@ public KeyValue next() throws IOException { 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(); @@ -83,6 +85,9 @@ public KeyValue next() throws IOException { 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]; @@ -91,7 +96,7 @@ public KeyValue next() throws IOException { } } unpacker.close(); - return new KeyValue(offset, keyBytes, Arrays.copyOfRange(mValue.getBytes(), 0, mValue.getLength())); + return new KeyValue(offset, keyBytes, Arrays.copyOfRange(mValue.getBytes(), 0, mValue.getLength()), timestamp); } else { return null; } @@ -131,25 +136,34 @@ public long getLength() throws IOException { @Override public void write(KeyValue keyValue) throws IOException { - byte[] kafkaKey = keyValue.getKafkaKey(); + 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 - // = 17 + N - ByteArrayOutputStream out = new ByteArrayOutputStream(17 + kafkaKey.length); + // = 27 + N + ByteArrayOutputStream out = new ByteArrayOutputStream(17 + timestampLength + kafkaKey.length); MessagePacker packer = MessagePack.newDefaultPacker(out) - .packMapHeader((kafkaKey.length == 0) ? 1 : 2) + .packMapHeader(numberOfFieldsMappedInHeader(keyValue)) .packInt(KAFKA_MESSAGE_OFFSET) .packLong(keyValue.getOffset()); - if (kafkaKey.length != 0) { + + 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); @@ -161,5 +175,17 @@ public void write(KeyValue keyValue) throws IOException { 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; + } } -} \ No newline at end of file +} diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index d29321fcf..090c5a544 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -92,7 +92,7 @@ public void write(ParsedMessage message) throws Exception { 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())); + 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/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java index cccdc8f70..130a74822 100644 --- a/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java +++ b/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java @@ -43,8 +43,14 @@ public void testMessagePackSequenceReadWriteRoundTrip() throws Exception { 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}, 1496318250)); + KeyValue kv4 = (new KeyValue(23235, null, new byte[]{23, 45, 40 ,10, 122}, 1496318250)); + fileWriter.write(kv1); fileWriter.write(kv2); + fileWriter.write(kv3); + fileWriter.write(kv4); + fileWriter.close(); FileReader fileReader = factory.BuildFileReader(tempLogFilePath, null); @@ -56,7 +62,17 @@ public void testMessagePackSequenceReadWriteRoundTrip() throws Exception { 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 From c23ce246257569a81ad8beb578ec611ea7d85f8c Mon Sep 17 00:00:00 2001 From: Vlad Ki Date: Sat, 8 Jul 2017 18:41:01 +0300 Subject: [PATCH 302/330] pom.xml: bump aws sdk to 1.11.160 --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 267be9906..5517424f7 100644 --- a/pom.xml +++ b/pom.xml @@ -100,12 +100,12 @@ com.amazonaws aws-java-sdk-s3 - 1.10.68 + 1.11.160 com.amazonaws aws-java-sdk-sts - 1.10.68 + 1.11.160 net.java.dev.jets3t From b4e68377e21924cf449da7626a25b22217452760 Mon Sep 17 00:00:00 2001 From: Vlad Ki Date: Sat, 8 Jul 2017 19:39:53 +0300 Subject: [PATCH 303/330] rebuild Iso8601MessageParser on top of TimestampedMessageParser --- README.md | 4 +- .../secor/parser/Iso8601MessageParser.java | 38 +++++++++---------- .../secor/parser/Iso8601ParserTest.java | 36 ++++++++++++------ 3 files changed, 45 insertions(+), 33 deletions(-) diff --git a/README.md b/README.md index 275aae312..682d1c7c3 100644 --- a/README.md +++ b/README.md @@ -62,7 +62,9 @@ One of the convenience features of Secor is the ability to group messages and sa - **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 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. +- **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. diff --git a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java index 6b18215d2..9a61e3105 100644 --- a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java @@ -24,7 +24,6 @@ import org.slf4j.LoggerFactory; import javax.xml.bind.DatatypeConverter; -import java.text.SimpleDateFormat; import java.util.Date; /** @@ -33,37 +32,34 @@ * @author Jurriaan Pruis (email@jurriaanpruis.nl) * */ -public class Iso8601MessageParser extends MessageParser { - private static final Logger LOG = LoggerFactory.getLogger(Iso8601MessageParser.class); - protected static final String defaultDate = "dt=1970-01-01"; - protected static final String defaultFormatter = "yyyy-MM-dd"; - protected static final SimpleDateFormat outputFormatter = new SimpleDateFormat(defaultFormatter); +public class Iso8601MessageParser extends TimestampedMessageParser { + private final boolean m_timestampRequired; public Iso8601MessageParser(SecorConfig config) { super(config); - outputFormatter.setTimeZone(config.getTimeZone()); + m_timestampRequired = config.isMessageTimestampRequired(); } @Override - public String[] extractPartitions(Message message) throws Exception { + public long extractTimestampMillis(final Message message) { JSONObject jsonObject = (JSONObject) JSONValue.parse(message.getPayload()); - String result[] = { defaultDate }; + Object fieldValue = jsonObject != null ? getJsonFieldValue(jsonObject) : null; - if (jsonObject != null) { - Object fieldValue = getJsonFieldValue(jsonObject); - if (fieldValue == null) { - LOG.warn("Missing field value. Using default partition = {}", defaultDate); - } else { - try { - Date dateFormat = DatatypeConverter.parseDateTime(fieldValue.toString()).getTime(); - result[0] = "dt=" + outputFormatter.format(dateFormat); - } catch (Exception e) { - LOG.warn("Impossible to convert date = {} as ISO-8601. Using date default = {}", - fieldValue.toString(), result[0]); + 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 result; + return 0; } } diff --git a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java index 1bd1dc056..67233a87c 100644 --- a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java @@ -43,8 +43,17 @@ public class Iso8601ParserTest extends TestCase { @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\"}" @@ -81,22 +90,27 @@ public void setUp() throws Exception { } @Test - public void testExtractDate() throws Exception { - Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); - assertEquals("dt=2014-07-30", new Iso8601MessageParser(mConfig).extractPartitions(mFormat1)[0]); - assertEquals("dt=2014-07-29", new Iso8601MessageParser(mConfig).extractPartitions(mFormat2)[0]); - assertEquals("dt=2001-07-04", new Iso8601MessageParser(mConfig).extractPartitions(mFormat3)[0]); - assertEquals("dt=2016-03-02", new Iso8601MessageParser(mConfig).extractPartitions(mFormat4)[0]); - assertEquals("dt=2006-01-02", new Iso8601MessageParser(mConfig).extractPartitions(mNanosecondISOFormat)[0]); - assertEquals("dt=1970-01-01", new Iso8601MessageParser(mConfig).extractPartitions(mInvalidDate)[0]); - assertEquals("dt=1970-01-01", new Iso8601MessageParser(mConfig).extractPartitions(mMissingDate)[0]); + 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 testNestedField() throws Exception { + public void testExtractNestedTimestampMillis() throws Exception { Mockito.when(mConfig.getMessageTimestampNameSeparator()).thenReturn("."); Mockito.when(mConfig.getMessageTimestampName()).thenReturn("meta_data.created"); - assertEquals("dt=2016-01-11", new Iso8601MessageParser(mConfig).extractPartitions(mNestedISOFormat)[0]); + Iso8601MessageParser parser = new Iso8601MessageParser(mConfig); + + assertEquals(1452513028647l, parser.getTimestampMillis(mNestedISOFormat)); } } From e88e9873ee00b7d366a7faff9968b356a2d9a246 Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Tue, 11 Jul 2017 11:32:21 +0530 Subject: [PATCH 304/330] Fix for backward compatibility in case of kafkaTimestamp usage --- pom.xml | 22 ++++++++++++++ src/main/config/secor.common.properties | 11 +++++-- .../pinterest/secor/common/KafkaClient.java | 7 ++++- .../pinterest/secor/common/SecorConfig.java | 4 +++ .../pinterest/secor/reader/MessageReader.java | 9 +++++- .../timestamp/Kafka10MessageTimestamp.java | 17 +++++++++++ .../timestamp/Kafka8MessageTimestamp.java | 17 +++++++++++ .../timestamp/KafkaMessageTimestamp.java | 11 +++++++ .../KafkaMessageTimestampFactory.java | 19 ++++++++++++ .../KafkaMessageTimestampFactoryTest.java | 29 +++++++++++++++++++ 10 files changed, 142 insertions(+), 4 deletions(-) create mode 100644 src/main/java/com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java create mode 100644 src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java create mode 100644 src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestamp.java create mode 100644 src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactory.java create mode 100644 src/test/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactoryTest.java diff --git a/pom.xml b/pom.xml index 5517424f7..792e229de 100644 --- a/pom.xml +++ b/pom.xml @@ -536,6 +536,14 @@ true + + org.apache.maven.plugins + maven-compiler-plugin + + 1.7 + 1.7 + + @@ -602,6 +610,20 @@ kafka-0.8-dev + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.0.2 + + + com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java + + + + + org.apache.kafka diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 2c0ce5d20..395ba1174 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -181,10 +181,17 @@ kafka.dual.commit.enabled=true # 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. +# 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. +# 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.Kafka8MessageTimestamp + # 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. diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 8a0add479..50482c602 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -18,6 +18,7 @@ import com.google.common.net.HostAndPort; import com.pinterest.secor.message.Message; +import com.pinterest.secor.timestamp.KafkaMessageTimestampFactory; import kafka.api.FetchRequestBuilder; import kafka.api.PartitionOffsetRequestInfo; import kafka.common.TopicAndPartition; @@ -50,10 +51,14 @@ public class KafkaClient { private SecorConfig mConfig; private ZookeeperConnector mZookeeperConnector; + private String mKafkaTimestampClassName; + private KafkaMessageTimestampFactory mKafkaMessageTimestampFactory; public KafkaClient(SecorConfig config) { mConfig = config; mZookeeperConnector = new ZookeeperConnector(mConfig); + mKafkaTimestampClassName = mConfig.getKafkaMessageTimestampClass(); + mKafkaMessageTimestampFactory = new KafkaMessageTimestampFactory(); } private HostAndPort findLeader(TopicPartition topicPartition) { @@ -143,7 +148,7 @@ private Message getMessage(TopicPartition topicPartition, long offset, } Long timestamp = null; if (mConfig.useKafkaTimestamp()) { - timestamp = messageAndOffset.message().timestamp(); + timestamp = mKafkaMessageTimestampFactory.create(mKafkaTimestampClassName).getTimestamp(messageAndOffset); } return new Message(topicPartition.getTopic(), topicPartition.getPartition(), messageAndOffset.offset(), keyBytes, payloadBytes, timestamp); diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index c1c3526a7..993a3163a 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -135,6 +135,10 @@ public boolean useKafkaTimestamp() { return getBoolean("kafka.useTimestamp", false); } + public String getKafkaMessageTimestampClass() { + return getString("kafka.message.timestamp.className"); + } + public int getGeneration() { return getInt("secor.generation"); } diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 871920c89..06792c703 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; @@ -58,6 +59,8 @@ public class MessageReader { protected final int mTopicPartitionForgetSeconds; protected final int mCheckMessagesPerSecond; protected int mNMessages; + protected String mKafkaMessageTimestampClassName; + protected KafkaMessageTimestampFactory mKafkaMessageTimestampFactory; public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws UnknownHostException { @@ -80,6 +83,8 @@ public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws StatsUtil.setLabel("secor.kafka.consumer.id", IdUtil.getConsumerId()); mTopicPartitionForgetSeconds = mConfig.getTopicPartitionForgetSeconds(); mCheckMessagesPerSecond = mConfig.getMessagesPerSecond() / mConfig.getConsumerThreads(); + mKafkaMessageTimestampClassName = mConfig.getKafkaMessageTimestampClass(); + mKafkaMessageTimestampFactory = new KafkaMessageTimestampFactory(); } private void updateAccessTime(TopicPartition topicPartition) { @@ -160,9 +165,11 @@ public Message read() { RateLimitUtil.acquire(mCheckMessagesPerSecond); } MessageAndMetadata kafkaMessage = mIterator.next(); + + Long timestamp = mKafkaMessageTimestampFactory.create(mKafkaMessageTimestampClassName).getTimestamp(kafkaMessage); Message message = new Message(kafkaMessage.topic(), kafkaMessage.partition(), kafkaMessage.offset(), kafkaMessage.key(), - kafkaMessage.message(), kafkaMessage.timestamp()); + kafkaMessage.message(), timestamp); TopicPartition topicPartition = new TopicPartition(message.getTopic(), message.getKafkaPartition()); updateAccessTime(topicPartition); 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..9e809d54b --- /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..447f50b46 --- /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 null; + } + + @Override + public Long getTimestamp(MessageAndOffset messageAndOffset) { + return null; + } +} 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..daef5b5e7 --- /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..41c0ed681 --- /dev/null +++ b/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactory.java @@ -0,0 +1,19 @@ +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); + + public static KafkaMessageTimestamp create(String kafkaTimestampClassName) { + KafkaMessageTimestamp kafkaMessageTimestamp = null; + try { + Class timestampClass = Class.forName(kafkaTimestampClassName); + kafkaMessageTimestamp = KafkaMessageTimestamp.class.cast(timestampClass.newInstance()); + } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { + LOG.error("Unable to create KafkaTimestampFactory", e); + } + return kafkaMessageTimestamp; + } +} 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..dca86a194 --- /dev/null +++ b/src/test/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactoryTest.java @@ -0,0 +1,29 @@ +package com.pinterest.secor.timestamp; + +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class KafkaMessageTimestampFactoryTest { + + private KafkaMessageTimestampFactory factory; + + @Before + public void setup() { + factory = new KafkaMessageTimestampFactory(); + } + + @Test + public void shouldReturnKafka8TimestampClassObject() { + Object timestamp = factory.create("com.pinterest.secor.timestamp.Kafka8MessageTimestamp"); + assertNotNull(timestamp); + assertEquals(timestamp.getClass(), Kafka8MessageTimestamp.class); + } + + @Test + public void shouldReturnNullForInvalidClass() { + Object timestamp = factory.create("com.pinterest.secor.timestamp.KafkaXXXMessageTimestamp"); + assertNull(timestamp); + } +} From 3b4f95288d8126977107a07d226cb9fdd73eca88 Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Tue, 11 Jul 2017 11:58:06 +0530 Subject: [PATCH 305/330] Fix for NPE --- src/main/java/com/pinterest/secor/message/ParsedMessage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/message/ParsedMessage.java b/src/main/java/com/pinterest/secor/message/ParsedMessage.java index d3d0bf65e..5c601117f 100644 --- a/src/main/java/com/pinterest/secor/message/ParsedMessage.java +++ b/src/main/java/com/pinterest/secor/message/ParsedMessage.java @@ -35,7 +35,7 @@ public String toString() { } public ParsedMessage(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload, - String[] mPartitions, long timestamp) { + String[] mPartitions, Long timestamp) { super(topic, kafkaPartition, offset, kafkaKey, payload, timestamp); this.mPartitions = mPartitions; } From b1906d105b851035b6140958cbef6c78b4ea3795 Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Tue, 11 Jul 2017 13:21:03 +0530 Subject: [PATCH 306/330] Trying fix NPE in setting timestamp --- src/main/java/com/pinterest/secor/reader/MessageReader.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 06792c703..0c05c7bc2 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -166,7 +166,10 @@ public Message read() { } MessageAndMetadata kafkaMessage = mIterator.next(); - Long timestamp = mKafkaMessageTimestampFactory.create(mKafkaMessageTimestampClassName).getTimestamp(kafkaMessage); + Long timestamp = null; + if (mConfig.useKafkaTimestamp()) { + timestamp = mKafkaMessageTimestampFactory.create(mKafkaMessageTimestampClassName).getTimestamp(kafkaMessage); + } Message message = new Message(kafkaMessage.topic(), kafkaMessage.partition(), kafkaMessage.offset(), kafkaMessage.key(), kafkaMessage.message(), timestamp); From 450e95e9e42da2c1e8bfd59a1ffdd8f33c9bf096 Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Tue, 11 Jul 2017 13:34:02 +0530 Subject: [PATCH 307/330] Changed long to Long in KeyValue.java to allow null fields for timestamp --- src/main/java/com/pinterest/secor/io/KeyValue.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/pinterest/secor/io/KeyValue.java b/src/main/java/com/pinterest/secor/io/KeyValue.java index 6dd60f3bd..93cc3045d 100644 --- a/src/main/java/com/pinterest/secor/io/KeyValue.java +++ b/src/main/java/com/pinterest/secor/io/KeyValue.java @@ -28,14 +28,14 @@ public class KeyValue { private final long mOffset; private final byte[] mKafkaKey; private final byte[] mValue; - private final long mTimestamp; + 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; + this.mTimestamp = -1l; } // constructor @@ -43,11 +43,11 @@ public KeyValue(long offset, byte[] kafkaKey, byte[] value) { this.mOffset = offset; this.mKafkaKey = kafkaKey; this.mValue = value; - this.mTimestamp = -1; + this.mTimestamp = -1l; } // constructor - public KeyValue(long offset, byte[] kafkaKey, byte[] value, long timestamp) { + public KeyValue(long offset, byte[] kafkaKey, byte[] value, Long timestamp) { this.mOffset = offset; this.mKafkaKey = kafkaKey; this.mValue = value; @@ -66,7 +66,7 @@ public byte[] getValue() { return this.mValue; } - public long getTimestamp() { + public Long getTimestamp() { return this.mTimestamp; } From 94270c872abee7cf58a2ff3c808caf273cfd9f1d Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Tue, 11 Jul 2017 13:44:46 +0530 Subject: [PATCH 308/330] Fixing failing test for MessagePackSequenceFileReaderWriterFactoryTest.java --- .../impl/MessagePackSequenceFileReaderWriterFactoryTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java index 130a74822..84bad994b 100644 --- a/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java +++ b/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java @@ -43,8 +43,8 @@ public void testMessagePackSequenceReadWriteRoundTrip() throws Exception { 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}, 1496318250)); - KeyValue kv4 = (new KeyValue(23235, null, new byte[]{23, 45, 40 ,10, 122}, 1496318250)); + 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); From c2cb770270b65cb63f664bb0d3a869975b34bb1a Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Fri, 14 Jul 2017 12:20:43 +0530 Subject: [PATCH 309/330] Moved dynamic class creation to constructor of timestamp factory and returning 0l in case of 0.8 timestamp --- src/main/config/secor.common.properties | 8 ++++---- .../com/pinterest/secor/common/KafkaClient.java | 9 ++------- .../com/pinterest/secor/reader/MessageReader.java | 9 ++------- .../secor/timestamp/Kafka8MessageTimestamp.java | 4 ++-- .../timestamp/KafkaMessageTimestampFactory.java | 14 +++++++++----- .../KafkaMessageTimestampFactoryTest.java | 14 ++++---------- 6 files changed, 23 insertions(+), 35 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 395ba1174..600060988 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -186,11 +186,11 @@ kafka.offsets.storage=zookeeper # in case you are enabling this parameter as `true`. kafka.useTimestamp=false -# Classname for the timestamp field you want to use. -# 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 +# 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.Kafka8MessageTimestamp +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. diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 50482c602..9017530ae 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -51,14 +51,12 @@ public class KafkaClient { private SecorConfig mConfig; private ZookeeperConnector mZookeeperConnector; - private String mKafkaTimestampClassName; private KafkaMessageTimestampFactory mKafkaMessageTimestampFactory; public KafkaClient(SecorConfig config) { mConfig = config; mZookeeperConnector = new ZookeeperConnector(mConfig); - mKafkaTimestampClassName = mConfig.getKafkaMessageTimestampClass(); - mKafkaMessageTimestampFactory = new KafkaMessageTimestampFactory(); + mKafkaMessageTimestampFactory = new KafkaMessageTimestampFactory(mConfig.getKafkaMessageTimestampClass()); } private HostAndPort findLeader(TopicPartition topicPartition) { @@ -146,10 +144,7 @@ private Message getMessage(TopicPartition topicPartition, long offset, payloadBytes = new byte[payload.limit()]; payload.get(payloadBytes); } - Long timestamp = null; - if (mConfig.useKafkaTimestamp()) { - timestamp = mKafkaMessageTimestampFactory.create(mKafkaTimestampClassName).getTimestamp(messageAndOffset); - } + Long timestamp = mKafkaMessageTimestampFactory.getKafkaMessageTimestamp().getTimestamp(messageAndOffset); return new Message(topicPartition.getTopic(), topicPartition.getPartition(), messageAndOffset.offset(), keyBytes, payloadBytes, timestamp); } diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index 0c05c7bc2..d0ef01ebf 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -59,7 +59,6 @@ public class MessageReader { protected final int mTopicPartitionForgetSeconds; protected final int mCheckMessagesPerSecond; protected int mNMessages; - protected String mKafkaMessageTimestampClassName; protected KafkaMessageTimestampFactory mKafkaMessageTimestampFactory; public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws @@ -83,8 +82,7 @@ public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws StatsUtil.setLabel("secor.kafka.consumer.id", IdUtil.getConsumerId()); mTopicPartitionForgetSeconds = mConfig.getTopicPartitionForgetSeconds(); mCheckMessagesPerSecond = mConfig.getMessagesPerSecond() / mConfig.getConsumerThreads(); - mKafkaMessageTimestampClassName = mConfig.getKafkaMessageTimestampClass(); - mKafkaMessageTimestampFactory = new KafkaMessageTimestampFactory(); + mKafkaMessageTimestampFactory = new KafkaMessageTimestampFactory(mConfig.getKafkaMessageTimestampClass()); } private void updateAccessTime(TopicPartition topicPartition) { @@ -166,10 +164,7 @@ public Message read() { } MessageAndMetadata kafkaMessage = mIterator.next(); - Long timestamp = null; - if (mConfig.useKafkaTimestamp()) { - timestamp = mKafkaMessageTimestampFactory.create(mKafkaMessageTimestampClassName).getTimestamp(kafkaMessage); - } + Long timestamp = mKafkaMessageTimestampFactory.getKafkaMessageTimestamp().getTimestamp(kafkaMessage); Message message = new Message(kafkaMessage.topic(), kafkaMessage.partition(), kafkaMessage.offset(), kafkaMessage.key(), kafkaMessage.message(), timestamp); diff --git a/src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java b/src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java index 447f50b46..f6f3dca09 100644 --- a/src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java +++ b/src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java @@ -7,11 +7,11 @@ public class Kafka8MessageTimestamp implements KafkaMessageTimestamp { @Override public Long getTimestamp(MessageAndMetadata kafkaMessage) { - return null; + return 0l; } @Override public Long getTimestamp(MessageAndOffset messageAndOffset) { - return null; + return 0l; } } diff --git a/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactory.java b/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactory.java index 41c0ed681..cc9a760cc 100644 --- a/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactory.java +++ b/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactory.java @@ -6,14 +6,18 @@ public class KafkaMessageTimestampFactory { private static final Logger LOG = LoggerFactory.getLogger(KafkaMessageTimestampFactory.class); - public static KafkaMessageTimestamp create(String kafkaTimestampClassName) { - KafkaMessageTimestamp kafkaMessageTimestamp = null; + private KafkaMessageTimestamp kafkaMessageTimestamp; + + public KafkaMessageTimestampFactory(String kafkaTimestampClassName) { try { Class timestampClass = Class.forName(kafkaTimestampClassName); - kafkaMessageTimestamp = KafkaMessageTimestamp.class.cast(timestampClass.newInstance()); + this.kafkaMessageTimestamp = KafkaMessageTimestamp.class.cast(timestampClass.newInstance()); } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { - LOG.error("Unable to create KafkaTimestampFactory", e); + throw new RuntimeException(e); } - return kafkaMessageTimestamp; + } + + public KafkaMessageTimestamp getKafkaMessageTimestamp() { + return this.kafkaMessageTimestamp; } } diff --git a/src/test/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactoryTest.java b/src/test/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactoryTest.java index dca86a194..1dec6cd76 100644 --- a/src/test/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactoryTest.java +++ b/src/test/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactoryTest.java @@ -1,6 +1,5 @@ package com.pinterest.secor.timestamp; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.*; @@ -9,21 +8,16 @@ public class KafkaMessageTimestampFactoryTest { private KafkaMessageTimestampFactory factory; - @Before - public void setup() { - factory = new KafkaMessageTimestampFactory(); - } - @Test public void shouldReturnKafka8TimestampClassObject() { - Object timestamp = factory.create("com.pinterest.secor.timestamp.Kafka8MessageTimestamp"); + factory = new KafkaMessageTimestampFactory("com.pinterest.secor.timestamp.Kafka8MessageTimestamp"); + Object timestamp = factory.getKafkaMessageTimestamp(); assertNotNull(timestamp); assertEquals(timestamp.getClass(), Kafka8MessageTimestamp.class); } - @Test + @Test(expected = RuntimeException.class) public void shouldReturnNullForInvalidClass() { - Object timestamp = factory.create("com.pinterest.secor.timestamp.KafkaXXXMessageTimestamp"); - assertNull(timestamp); + factory = new KafkaMessageTimestampFactory("com.pinterest.secor.timestamp.KafkaxxMessageTimestamp"); } } From 1c1ba5e48fbe09228667e19f1fffdee7d7537520 Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Fri, 14 Jul 2017 22:59:09 +0530 Subject: [PATCH 310/330] Long to long in timestamp classes --- .../pinterest/secor/timestamp/Kafka10MessageTimestamp.java | 4 ++-- .../com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java | 4 ++-- .../com/pinterest/secor/timestamp/KafkaMessageTimestamp.java | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/main/java/com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java b/src/main/java/com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java index 9e809d54b..043901ef5 100644 --- a/src/main/java/com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java +++ b/src/main/java/com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java @@ -6,12 +6,12 @@ public class Kafka10MessageTimestamp implements KafkaMessageTimestamp { @Override - public Long getTimestamp(MessageAndMetadata kafkaMessage) { + public long getTimestamp(MessageAndMetadata kafkaMessage) { return kafkaMessage.timestamp(); } @Override - public Long getTimestamp(MessageAndOffset messageAndOffset) { + 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 index f6f3dca09..bc461734e 100644 --- a/src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java +++ b/src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java @@ -6,12 +6,12 @@ public class Kafka8MessageTimestamp implements KafkaMessageTimestamp { @Override - public Long getTimestamp(MessageAndMetadata kafkaMessage) { + public long getTimestamp(MessageAndMetadata kafkaMessage) { return 0l; } @Override - public Long getTimestamp(MessageAndOffset messageAndOffset) { + 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 index daef5b5e7..4d53b13a9 100644 --- a/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestamp.java +++ b/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestamp.java @@ -5,7 +5,7 @@ public interface KafkaMessageTimestamp { - Long getTimestamp(MessageAndMetadata kafkaMessage); + long getTimestamp(MessageAndMetadata kafkaMessage); - Long getTimestamp(MessageAndOffset messageAndOffset); + long getTimestamp(MessageAndOffset messageAndOffset); } From 45a8f43bbb844ab308a99ce7c343504fb0ee61f8 Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Fri, 14 Jul 2017 23:46:02 +0530 Subject: [PATCH 311/330] Changed timestamp type from Long to long in Message class --- src/main/java/com/pinterest/secor/common/KafkaClient.java | 5 ++++- src/main/java/com/pinterest/secor/io/KeyValue.java | 8 ++++---- src/main/java/com/pinterest/secor/message/Message.java | 6 +++--- .../java/com/pinterest/secor/message/ParsedMessage.java | 2 +- .../java/com/pinterest/secor/reader/MessageReader.java | 4 +++- .../java/com/pinterest/secor/message/MessageTest.java | 2 +- .../com/pinterest/secor/parser/JsonMessageParserTest.java | 2 +- 7 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 9017530ae..82c7b86ee 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -144,7 +144,10 @@ private Message getMessage(TopicPartition topicPartition, long offset, payloadBytes = new byte[payload.limit()]; payload.get(payloadBytes); } - Long timestamp = mKafkaMessageTimestampFactory.getKafkaMessageTimestamp().getTimestamp(messageAndOffset); + long timestamp = (mConfig.useKafkaTimestamp()) + ? mKafkaMessageTimestampFactory.getKafkaMessageTimestamp().getTimestamp(messageAndOffset) + : 0l; + return new Message(topicPartition.getTopic(), topicPartition.getPartition(), messageAndOffset.offset(), keyBytes, payloadBytes, timestamp); } diff --git a/src/main/java/com/pinterest/secor/io/KeyValue.java b/src/main/java/com/pinterest/secor/io/KeyValue.java index 93cc3045d..9ca184c1a 100644 --- a/src/main/java/com/pinterest/secor/io/KeyValue.java +++ b/src/main/java/com/pinterest/secor/io/KeyValue.java @@ -28,14 +28,14 @@ public class KeyValue { private final long mOffset; private final byte[] mKafkaKey; private final byte[] mValue; - private final Long mTimestamp; + private final long mTimestamp; // constructor public KeyValue(long offset, byte[] value) { this.mOffset = offset; this.mKafkaKey = new byte[0]; this.mValue = value; - this.mTimestamp = -1l; + this.mTimestamp = -1; } // constructor @@ -43,7 +43,7 @@ public KeyValue(long offset, byte[] kafkaKey, byte[] value) { this.mOffset = offset; this.mKafkaKey = kafkaKey; this.mValue = value; - this.mTimestamp = -1l; + this.mTimestamp = -1; } // constructor @@ -66,7 +66,7 @@ public byte[] getValue() { return this.mValue; } - public Long getTimestamp() { + public long getTimestamp() { return this.mTimestamp; } diff --git a/src/main/java/com/pinterest/secor/message/Message.java b/src/main/java/com/pinterest/secor/message/Message.java index 384336313..f355325ed 100644 --- a/src/main/java/com/pinterest/secor/message/Message.java +++ b/src/main/java/com/pinterest/secor/message/Message.java @@ -34,7 +34,7 @@ public class Message { private long mOffset; private byte[] mKafkaKey; private byte[] mPayload; - private Long mTimestamp; + private long mTimestamp; protected String fieldsToString() { return "topic='" + mTopic + '\'' + @@ -50,7 +50,7 @@ public String toString() { return "Message{" + fieldsToString() + '}'; } - public Message(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload, Long timestamp) { + public Message(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload, long timestamp) { mTopic = topic; mKafkaPartition = kafkaPartition; mOffset = offset; @@ -85,7 +85,7 @@ public byte[] getPayload() { return mPayload; } - public Long getTimestamp() { + public long getTimestamp() { return mTimestamp; } diff --git a/src/main/java/com/pinterest/secor/message/ParsedMessage.java b/src/main/java/com/pinterest/secor/message/ParsedMessage.java index 5c601117f..d3d0bf65e 100644 --- a/src/main/java/com/pinterest/secor/message/ParsedMessage.java +++ b/src/main/java/com/pinterest/secor/message/ParsedMessage.java @@ -35,7 +35,7 @@ public String toString() { } public ParsedMessage(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload, - String[] mPartitions, Long timestamp) { + String[] mPartitions, long timestamp) { super(topic, kafkaPartition, offset, kafkaKey, payload, timestamp); this.mPartitions = mPartitions; } diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index d0ef01ebf..b99813522 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -164,7 +164,9 @@ public Message read() { } MessageAndMetadata kafkaMessage = mIterator.next(); - Long timestamp = mKafkaMessageTimestampFactory.getKafkaMessageTimestamp().getTimestamp(kafkaMessage); + long timestamp = (mConfig.useKafkaTimestamp()) + ? mKafkaMessageTimestampFactory.getKafkaMessageTimestamp().getTimestamp(kafkaMessage) + : 0l; Message message = new Message(kafkaMessage.topic(), kafkaMessage.partition(), kafkaMessage.offset(), kafkaMessage.key(), kafkaMessage.message(), timestamp); diff --git a/src/test/java/com/pinterest/secor/message/MessageTest.java b/src/test/java/com/pinterest/secor/message/MessageTest.java index 50191b02a..d4eb8c59b 100644 --- a/src/test/java/com/pinterest/secor/message/MessageTest.java +++ b/src/test/java/com/pinterest/secor/message/MessageTest.java @@ -6,7 +6,7 @@ public class MessageTest { @Test public void testNullPayload() { - Message message = new Message("testTopic", 0, 123, null, null, null); + 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 diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index 01b3af3eb..cdff37f55 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -70,7 +70,7 @@ public void setUp() throws Exception { 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, null, messageWithoutTimestamp, null); + 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"); From c31468478f61ff27df144a7915858eefbce017b5 Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Sat, 15 Jul 2017 07:58:01 +0530 Subject: [PATCH 312/330] Convert Long to long in KeyValue constructor --- src/main/java/com/pinterest/secor/io/KeyValue.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/io/KeyValue.java b/src/main/java/com/pinterest/secor/io/KeyValue.java index 9ca184c1a..6dd60f3bd 100644 --- a/src/main/java/com/pinterest/secor/io/KeyValue.java +++ b/src/main/java/com/pinterest/secor/io/KeyValue.java @@ -47,7 +47,7 @@ public KeyValue(long offset, byte[] kafkaKey, byte[] value) { } // constructor - public KeyValue(long offset, byte[] kafkaKey, byte[] value, Long timestamp) { + public KeyValue(long offset, byte[] kafkaKey, byte[] value, long timestamp) { this.mOffset = offset; this.mKafkaKey = kafkaKey; this.mValue = value; From a0764ecd65a0428e2b33ced943214bca6c368e8a Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Sun, 16 Jul 2017 10:42:55 +0530 Subject: [PATCH 313/330] Modified README.md for current default active profile --- README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 682d1c7c3..c8383bc94 100644 --- a/README.md +++ b/README.md @@ -29,13 +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.8 profile) +# 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.10 client you should use the kafka-0.10-dev profile -mvn -Pkafka-0.10-dev package +# 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) From 2dfd2f7f40dcefd1d42ec21bf4ee0ed4b2b4de09 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Sun, 16 Jul 2017 17:29:24 -0700 Subject: [PATCH 314/330] Bump up version to v0.24 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 792e229de..cb2370775 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.24-SNAPSHOT + 0.24 jar secor Kafka to s3/gs/swift logs exporter From 77200c8ca3eb9e4ff6ad6fe4925b74615d7e0fbf Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Sun, 16 Jul 2017 17:40:15 -0700 Subject: [PATCH 315/330] Switch to 0.25-SNAPSHOT Default release build is now using 0.10 --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index cb2370775..31d08512e 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.24 + 0.25-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter @@ -598,7 +598,7 @@ org.apache.kafka kafka_2.10 - 0.8.2.1 + 0.10.2.0 org.slf4j From ca092d9250f55551acabb203395c8480ca1e7c9a Mon Sep 17 00:00:00 2001 From: Jeff Pollard Date: Thu, 27 Jul 2017 18:01:03 -0700 Subject: [PATCH 316/330] Catch no committed message --- .../pinterest/secor/common/KafkaClient.java | 21 +++++++++++++++++-- .../secor/tools/ProgressMonitor.java | 10 +++++++-- 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 82c7b86ee..70f3e659e 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -59,6 +59,8 @@ public KafkaClient(SecorConfig config) { mKafkaMessageTimestampFactory = new KafkaMessageTimestampFactory(mConfig.getKafkaMessageTimestampClass()); } + public class MessageDoesNotExistException extends RuntimeException {} + private HostAndPort findLeader(TopicPartition topicPartition) { SimpleConsumer consumer = null; try { @@ -127,8 +129,13 @@ 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 == 1) { + throw new MessageDoesNotExistException(); + } else { + throw new RuntimeException("Error fetching offset data. Reason: " + errorCode); + } } MessageAndOffset messageAndOffset = response.messageSet( topicPartition.getTopic(), topicPartition.getPartition()).iterator().next(); @@ -212,6 +219,16 @@ public Message getCommittedMessage(TopicPartition topicPartition) throws Excepti } 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 + return null; } finally { if (consumer != null) { consumer.close(); diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index aef5b0098..24c12e905 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -207,8 +207,14 @@ private List getStats() throws Exception { assert committedOffset <= lastOffset: Long.toString(committedOffset) + " <= " + lastOffset; - long offsetLag = lastOffset - committedOffset; - long timestampMillisLag = lastTimestampMillis - committedTimestampMillis; + long offsetLag = 0L; + long timestampMillisLag = 0L; + + if (committedMessage != null) { + offsetLag = lastOffset - committedOffset; + timestampMillisLag = lastTimestampMillis - committedTimestampMillis; + } + Map tags = ImmutableMap.of( Stat.STAT_KEYS.TOPIC.getName(), topic, Stat.STAT_KEYS.PARTITION.getName(), Integer.toString(partition), From 2067302a37aaf54cc1d18d520ee6ea3997aa9198 Mon Sep 17 00:00:00 2001 From: Jeff Pollard Date: Fri, 28 Jul 2017 14:24:50 -0700 Subject: [PATCH 317/330] Match to built in errors enum --- src/main/java/com/pinterest/secor/common/KafkaClient.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index 70f3e659e..a06b35b56 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -31,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; @@ -131,7 +132,7 @@ private Message getMessage(TopicPartition topicPartition, long offset, consumer.close(); int errorCode = response.errorCode(topicPartition.getTopic(), topicPartition.getPartition()); - if (errorCode == 1) { + if (errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { throw new MessageDoesNotExistException(); } else { throw new RuntimeException("Error fetching offset data. Reason: " + errorCode); From 2dbf16a187c0e5de4e38c7d295eea981bc0869fb Mon Sep 17 00:00:00 2001 From: Jeff Pollard Date: Fri, 28 Jul 2017 14:26:07 -0700 Subject: [PATCH 318/330] Log a warning if getCommittedMessage is unable to find a message --- src/main/java/com/pinterest/secor/common/KafkaClient.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java index a06b35b56..1e1a20ae9 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -229,6 +229,7 @@ public Message getCommittedMessage(TopicPartition topicPartition) throws Excepti // 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) { From 4de6fec463662e13ae7bd57c73721c47c65b055e Mon Sep 17 00:00:00 2001 From: Jeff Pollard Date: Fri, 28 Jul 2017 14:27:37 -0700 Subject: [PATCH 319/330] Simply skip reporting stats if there is no latest message --- .../com/pinterest/secor/tools/ProgressMonitor.java | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 24c12e905..4d44d3bf3 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -193,6 +193,7 @@ private List getStats() throws Exception { long committedTimestampMillis = -1; if (committedMessage == null) { LOG.warn("no committed message found in topic {} partition {}", topic, partition); + continue; } else { committedOffset = committedMessage.getOffset(); committedTimestampMillis = getTimestamp(committedMessage); @@ -207,14 +208,8 @@ private List getStats() throws Exception { assert committedOffset <= lastOffset: Long.toString(committedOffset) + " <= " + lastOffset; - long offsetLag = 0L; - long timestampMillisLag = 0L; - - if (committedMessage != null) { - offsetLag = lastOffset - committedOffset; - timestampMillisLag = lastTimestampMillis - committedTimestampMillis; - } - + long offsetLag = lastOffset - committedOffset; + long timestampMillisLag = lastTimestampMillis - committedTimestampMillis; Map tags = ImmutableMap.of( Stat.STAT_KEYS.TOPIC.getName(), topic, Stat.STAT_KEYS.PARTITION.getName(), Integer.toString(partition), From 97b5b1174c5f67e650b65b9d8e9c9e11ae5f52a5 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Wed, 20 Sep 2017 23:51:20 -0700 Subject: [PATCH 320/330] Bump the version to 0.25 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 31d08512e..e58af2102 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.25-SNAPSHOT + 0.25 jar secor Kafka to s3/gs/swift logs exporter From 68d700e66e4a4e0bf3c03e26334e16d46ac11b86 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 21 Sep 2017 00:05:10 -0700 Subject: [PATCH 321/330] Set the version to 0.26-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e58af2102..67eee6df5 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.pinterest secor - 0.25 + 0.26-SNAPSHOT jar secor Kafka to s3/gs/swift logs exporter From 75fe0cf454ba28293850aa5649467f3414c3cbd7 Mon Sep 17 00:00:00 2001 From: Henry Cai Date: Thu, 21 Sep 2017 14:03:03 -0700 Subject: [PATCH 322/330] Per https://github.com/travis-ci/travis-ci/issues/7884, oraclejdk7 is no longer supported, move to openjdk7 --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 50f212ded..1f45b397e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -5,7 +5,7 @@ addons: env: - PATH=$PATH:$HOME/.s3cmd SECOR_LOCAL_S3=true S3CMD=1.0.1 jdk: - - oraclejdk7 + - openjdk7 - oraclejdk8 before_install: - wget https://github.com/s3tools/s3cmd/archive/v$S3CMD.tar.gz -O /tmp/s3cmd.tar.gz From c4d1811cbb07d7fe919f64ebdeffac45e3dfe13d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Tom=C3=A1s=20Senart?= Date: Wed, 11 Oct 2017 11:14:17 +0200 Subject: [PATCH 323/330] Document Microsoft Azure Blob Storage support This commit documents the Microsoft Azure Blob Storage support introduced in PR #194. --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index c8383bc94..6886f6432 100644 --- a/README.md +++ b/README.md @@ -2,7 +2,7 @@ [![Build Status](https://travis-ci.org/pinterest/secor.svg)](https://travis-ci.org/pinterest/secor) -Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud Storage] and [Openstack Swift]. +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, @@ -186,6 +186,7 @@ If you have any questions or comments, you can reach us at [secor-users@googlegr [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 From c96cee4f1b6cc6a7335ca524675a28168dde9311 Mon Sep 17 00:00:00 2001 From: Jethro Carr Date: Thu, 12 Oct 2017 23:39:25 +1300 Subject: [PATCH 324/330] Use official Docker image "openjdk" instead of "java" The official Java image is deprecated and has no longer recieved any updates since Dec 31 2016. As the Java image has always been built around OpenJDK, using the official Dockerhub OpenJDK image is a logical replacement. See comments at: https://hub.docker.com/r/library/java/ --- Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Dockerfile b/Dockerfile index 9a47a3b42..6504267b0 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,4 +1,4 @@ -FROM java:8 +FROM openjdk:8 RUN mkdir -p /opt/secor ADD target/secor-*-bin.tar.gz /opt/secor/ From 53901e73152565e9dd565023ec273dc9b95b93f4 Mon Sep 17 00:00:00 2001 From: Atilla Genc Date: Fri, 13 Oct 2017 16:15:27 -0700 Subject: [PATCH 325/330] #369 fix handling null values for orc file generation --- .../java/com/pinterest/secor/util/orc/VectorColumnFiller.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/util/orc/VectorColumnFiller.java b/src/main/java/com/pinterest/secor/util/orc/VectorColumnFiller.java index ea49203a6..73284f496 100644 --- a/src/main/java/com/pinterest/secor/util/orc/VectorColumnFiller.java +++ b/src/main/java/com/pinterest/secor/util/orc/VectorColumnFiller.java @@ -218,7 +218,7 @@ public static void fillRow(int rowIndex, JsonConverter[] converters, JsonElement field = data.get(fieldNames.get(c)); if (field == null) { batch.cols[c].noNulls = false; - batch.cols[c].isNull[batch.size] = true; + batch.cols[c].isNull[0] = true; } else { converters[c].convert(field, batch.cols[c], rowIndex); } From e3827c43fb8ba9f2d54e8041d7da0d967a292c24 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Pedro?= Date: Wed, 29 Nov 2017 15:48:28 -0200 Subject: [PATCH 326/330] Adding custom topic name implementation --- src/main/config/secor.common.properties | 2 + .../pinterest/secor/common/SecorConfig.java | 4 ++ .../secor/uploader/HadoopS3UploadManager.java | 28 ++++++++++- .../secor/uploader/UploaderTest.java | 48 +++++++++++++++++++ 4 files changed, 81 insertions(+), 1 deletion(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 600060988..93860ba30 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -20,6 +20,8 @@ # Regular expression matching names of consumed topics. secor.kafka.topic_filter=.* secor.kafka.topic_blacklist= +# Pattern example of customtopicsnames: topicName1:customTopicName1;topicName2:customTopicName2; +secor.kafka.customtopicsnames= # Choose what to fill according to the service you are using # in the choice option you can fill S3, GS, Swift or Azure diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 993a3163a..fe8a38679 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -578,6 +578,10 @@ public String getMetricsCollectorClass() { return getString("secor.monitoring.metrics.collector.class"); } + public String getCustomTopicsNames() { + return getString("secor.kafka.customtopicsnames"); + } + /** * This method is used for fetching all the properties which start with the given prefix. * It returns a Map of all those key-val. diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java index 166795d1a..b10a9cbb1 100644 --- a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -42,12 +42,31 @@ public class HadoopS3UploadManager extends UploadManager { public HadoopS3UploadManager(SecorConfig config) { super(config); } + + private Map getCustomTopicsNamesMap(SecorConfig mConfig) { + Map customTopicsNamesMap = null; + if(mConfig.getCustomTopicsNames() != null) { + String customTopicsNamesField = mConfig.getCustomTopicsNames(); + if (customTopicsNamesField.length() != 0 && customTopicsNamesField.contains(";")) { + String[] customTopicsNames = customTopicsNamesField.split(";"); + customTopicsNamesMap = new HashMap(); + for (String topicNames: customTopicsNames) { + if (topicNames.matches("[A-Za-z0-9_]+:[A-Za-z0-9_]+")) { + customTopicsNamesMap.put(topicNames.split(":")[0], topicNames.split(":")[1]); + } + } + } + } + return customTopicsNamesMap; + } public Handle upload(LogFilePath localPath) throws Exception { String prefix = FileUtil.getPrefix(localPath.getTopic(), mConfig); LogFilePath path = localPath.withPrefix(prefix); final String localLogFilename = localPath.getLogFilePath(); final String logFileName; + final String topicName = localPath.getTopic(); + final Map customTopicsNamesMap = getCustomTopicsNamesMap(mConfig); if (FileUtil.s3PathPrefixIsAltered(path.getLogFilePath(), mConfig)) { logFileName = localPath.withPrefix(FileUtil.getS3AlternativePrefix(mConfig)).getLogFilePath(); @@ -63,7 +82,14 @@ public Handle upload(LogFilePath localPath) throws Exception { @Override public void run() { try { - FileUtil.moveToCloud(localLogFilename, logFileName); + if (customTopicsNamesMap != null) { + String logFileNameCustomTopicName = logFileName; + logFileNameCustomTopicName = logFileNameCustomTopicName.replace(topicName, + customTopicsNamesMap.get(topicName)); + FileUtil.moveToCloud(localLogFilename, logFileNameCustomTopicName); + }else { + FileUtil.moveToCloud(localLogFilename, logFileName); + } } catch (IOException e) { throw new RuntimeException(e); } diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index 4ae220ccf..3152fb216 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -197,6 +197,54 @@ public void testUploadFiles() throws Exception { 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"); + Mockito.when(mConfig.getCustomTopicsNames()).thenReturn("some_topic:some_topic_test;"); + + 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_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).unlock(lockPath); + } public void testDeleteTopicPartition() throws Exception { Mockito.when( From 50758c8adb40da0904e14fef28f69667119c441b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Pedro?= Date: Mon, 4 Dec 2017 14:04:12 -0200 Subject: [PATCH 327/330] Style fixes --- .../com/pinterest/secor/uploader/HadoopS3UploadManager.java | 2 +- src/main/java/com/pinterest/secor/uploader/Uploader.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java index b10a9cbb1..6f2ef22ca 100644 --- a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -87,7 +87,7 @@ public void run() { logFileNameCustomTopicName = logFileNameCustomTopicName.replace(topicName, customTopicsNamesMap.get(topicName)); FileUtil.moveToCloud(localLogFilename, logFileNameCustomTopicName); - }else { + } else { FileUtil.moveToCloud(localLogFilename, logFileName); } } catch (IOException e) { diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index 0c9048743..a9d1e6b3c 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -257,4 +257,4 @@ public void applyPolicy() throws Exception { checkTopicPartition(topicPartition); } } -} +} \ No newline at end of file From 99223addeffb0be8d658b758c00b6213420eab4c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Pedro?= Date: Mon, 4 Dec 2017 16:09:12 -0200 Subject: [PATCH 328/330] Adding custom topics names on LogFilePath --- src/main/config/secor.common.properties | 4 +-- .../pinterest/secor/common/LogFilePath.java | 34 +++++++++++++++--- .../pinterest/secor/common/SecorConfig.java | 4 +-- .../secor/uploader/HadoopS3UploadManager.java | 35 +++---------------- .../secor/uploader/UploaderTest.java | 6 +++- 5 files changed, 44 insertions(+), 39 deletions(-) diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties index 93860ba30..5810d979d 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -20,8 +20,8 @@ # Regular expression matching names of consumed topics. secor.kafka.topic_filter=.* secor.kafka.topic_blacklist= -# Pattern example of customtopicsnames: topicName1:customTopicName1;topicName2:customTopicName2; -secor.kafka.customtopicsnames= +# 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 diff --git a/src/main/java/com/pinterest/secor/common/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java index 6f0e8b14c..3d2949dbd 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -25,6 +25,7 @@ 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 @@ -53,6 +54,7 @@ public class LogFilePath { private final int[] mKafkaPartitions; private final long[] mOffsets; private final String mExtension; + private Map customTopicsNames; private MessageDigest messageDigest; @@ -145,16 +147,22 @@ public LogFilePath withPrefix(String prefix) { } public String getLogFileParentDir() { - ArrayList elements = new ArrayList(); + ArrayList elements = new ArrayList(); if (mPrefix != null && mPrefix.length() > 0) { elements.add(mPrefix); } if (mTopic != null && mTopic.length() > 0) { - elements.add(mTopic); + if (getCustomTopicsNames() != null && + getCustomTopicsNames().isEmpty() == false && + 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()); @@ -189,7 +197,7 @@ private String getLogFileBasename() { } return StringUtils.join(basenameElements, "_"); } - + public String getLogFilePath() { String basename = getLogFileBasename(); @@ -200,6 +208,16 @@ public String getLogFilePath() { return StringUtils.join(pathElements, "/") + mExtension; } + public String getLogFilePath(Map customTopicsNamesMap) { + String basename = getLogFileBasename(); + setCustomTopicsNames(customTopicsNamesMap); + ArrayList pathElements = new ArrayList(); + pathElements.add(getLogFileDir()); + pathElements.add(basename); + + return StringUtils.join(pathElements, "/") + mExtension; + } + public String getLogFileCrcPath() { String basename = "." + getLogFileBasename() + ".crc"; @@ -243,6 +261,14 @@ public long[] getOffsets() { public String getExtension() { return mExtension; } + + private Map getCustomTopicsNames() { + return customTopicsNames; + } + + private void setCustomTopicsNames(Map customTopicsNames) { + this.customTopicsNames = customTopicsNames; + } @Override public boolean equals(Object o) { diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index fe8a38679..2dd6d1ca0 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -578,8 +578,8 @@ public String getMetricsCollectorClass() { return getString("secor.monitoring.metrics.collector.class"); } - public String getCustomTopicsNames() { - return getString("secor.kafka.customtopicsnames"); + public Map getCustomTopicsNames() { + return getPropertyMapForPrefix("secor.kafka.customtopicname"); } /** diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java index 6f2ef22ca..706414f1f 100644 --- a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -42,38 +42,20 @@ public class HadoopS3UploadManager extends UploadManager { public HadoopS3UploadManager(SecorConfig config) { super(config); } - - private Map getCustomTopicsNamesMap(SecorConfig mConfig) { - Map customTopicsNamesMap = null; - if(mConfig.getCustomTopicsNames() != null) { - String customTopicsNamesField = mConfig.getCustomTopicsNames(); - if (customTopicsNamesField.length() != 0 && customTopicsNamesField.contains(";")) { - String[] customTopicsNames = customTopicsNamesField.split(";"); - customTopicsNamesMap = new HashMap(); - for (String topicNames: customTopicsNames) { - if (topicNames.matches("[A-Za-z0-9_]+:[A-Za-z0-9_]+")) { - customTopicsNamesMap.put(topicNames.split(":")[0], topicNames.split(":")[1]); - } - } - } - } - return customTopicsNamesMap; - } public Handle upload(LogFilePath localPath) throws Exception { String prefix = FileUtil.getPrefix(localPath.getTopic(), mConfig); LogFilePath path = localPath.withPrefix(prefix); final String localLogFilename = localPath.getLogFilePath(); final String logFileName; - final String topicName = localPath.getTopic(); - final Map customTopicsNamesMap = getCustomTopicsNamesMap(mConfig); - + if (FileUtil.s3PathPrefixIsAltered(path.getLogFilePath(), mConfig)) { - logFileName = localPath.withPrefix(FileUtil.getS3AlternativePrefix(mConfig)).getLogFilePath(); + logFileName = localPath.withPrefix(FileUtil.getS3AlternativePrefix(mConfig)) + .getLogFilePath(mConfig.getCustomTopicsNames()); LOG.info("Will upload file to alternative s3 prefix path {}", logFileName); } else { - logFileName = path.getLogFilePath(); + logFileName = path.getLogFilePath(mConfig.getCustomTopicsNames()); } LOG.info("uploading file {} to {}", localLogFilename, logFileName); @@ -82,14 +64,7 @@ public Handle upload(LogFilePath localPath) throws Exception { @Override public void run() { try { - if (customTopicsNamesMap != null) { - String logFileNameCustomTopicName = logFileName; - logFileNameCustomTopicName = logFileNameCustomTopicName.replace(topicName, - customTopicsNamesMap.get(topicName)); - FileUtil.moveToCloud(localLogFilename, logFileNameCustomTopicName); - } else { - FileUtil.moveToCloud(localLogFilename, logFileName); - } + FileUtil.moveToCloud(localLogFilename, logFileName); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index 3152fb216..335afee9f 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -39,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. @@ -218,7 +220,9 @@ public void testUploadFilesCustomTopicName() throws Exception{ Mockito.when(mConfig.getCloudService()).thenReturn("S3"); Mockito.when(mConfig.getS3Bucket()).thenReturn("some_bucket"); Mockito.when(mConfig.getS3Path()).thenReturn("some_s3_parent_dir"); - Mockito.when(mConfig.getCustomTopicsNames()).thenReturn("some_topic:some_topic_test;"); + Map customTopicsNamesMap = new HashMap(); + customTopicsNamesMap.put("some_topic", "some_topic_test"); + Mockito.when(mConfig.getCustomTopicsNames()).thenReturn(customTopicsNamesMap); HashSet logFilePaths = new HashSet(); logFilePaths.add(mLogFilePath); From b2c58868dda8e76ab66636d18b17a09f0f8b39a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Pedro?= Date: Tue, 5 Dec 2017 11:38:22 -0200 Subject: [PATCH 329/330] Fix set method to custom topic name --- .../com/pinterest/secor/common/LogFilePath.java | 14 ++------------ .../secor/uploader/HadoopS3UploadManager.java | 5 +++-- 2 files changed, 5 insertions(+), 14 deletions(-) diff --git a/src/main/java/com/pinterest/secor/common/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java index 3d2949dbd..f677e6440 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -208,16 +208,6 @@ public String getLogFilePath() { return StringUtils.join(pathElements, "/") + mExtension; } - public String getLogFilePath(Map customTopicsNamesMap) { - String basename = getLogFileBasename(); - setCustomTopicsNames(customTopicsNamesMap); - ArrayList pathElements = new ArrayList(); - pathElements.add(getLogFileDir()); - pathElements.add(basename); - - return StringUtils.join(pathElements, "/") + mExtension; - } - public String getLogFileCrcPath() { String basename = "." + getLogFileBasename() + ".crc"; @@ -262,11 +252,11 @@ public String getExtension() { return mExtension; } - private Map getCustomTopicsNames() { + public Map getCustomTopicsNames() { return customTopicsNames; } - private void setCustomTopicsNames(Map customTopicsNames) { + public void setCustomTopicsNames(Map customTopicsNames) { this.customTopicsNames = customTopicsNames; } diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java index 706414f1f..32662fd92 100644 --- a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -46,16 +46,17 @@ public HadoopS3UploadManager(SecorConfig 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(mConfig.getCustomTopicsNames()); + .getLogFilePath(); LOG.info("Will upload file to alternative s3 prefix path {}", logFileName); } else { - logFileName = path.getLogFilePath(mConfig.getCustomTopicsNames()); + logFileName = path.getLogFilePath(); } LOG.info("uploading file {} to {}", localLogFilename, logFileName); From 8b9b1a77a018cee5d0e6ad0d5c6bf75831d7a68e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Pedro?= Date: Thu, 7 Dec 2017 13:43:22 -0200 Subject: [PATCH 330/330] Fix bug on custom topic name --- src/main/java/com/pinterest/secor/common/LogFilePath.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/pinterest/secor/common/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java index f677e6440..e071b3d91 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -154,7 +154,8 @@ public String getLogFileParentDir() { if (mTopic != null && mTopic.length() > 0) { if (getCustomTopicsNames() != null && getCustomTopicsNames().isEmpty() == false && - getCustomTopicsNames().get(mTopic).length() > 0) { + getCustomTopicsNames().get(mTopic) != null && + getCustomTopicsNames().get(mTopic).length() > 0) { elements.add(getCustomTopicsNames().get(mTopic)); } else { elements.add(mTopic);