├── .gitignore ├── yuvi ├── src │ ├── main │ │ ├── java │ │ │ └── com │ │ │ │ └── pinterest │ │ │ │ └── yuvi │ │ │ │ ├── bitstream │ │ │ │ ├── ParseException.java │ │ │ │ ├── BitStreamIterator.java │ │ │ │ └── BitStream.java │ │ │ │ ├── writer │ │ │ │ ├── MetricWriter.java │ │ │ │ ├── kafka │ │ │ │ │ ├── MetricsConsumerRebalanceListener.java │ │ │ │ │ ├── MetricsWriterTask.java │ │ │ │ │ ├── ThriftTextMessageDeserializer.java │ │ │ │ │ └── KafkaMetricWriter.java │ │ │ │ └── FileMetricWriter.java │ │ │ │ ├── metricstore │ │ │ │ ├── TimeSeriesIterator.java │ │ │ │ ├── MetricStore.java │ │ │ │ ├── CachingVarBitTimeSeriesIterator.java │ │ │ │ ├── VarBitMetricStore.java │ │ │ │ ├── offheap │ │ │ │ │ └── OffHeapVarBitMetricStore.java │ │ │ │ └── VarBitTimeSeries.java │ │ │ │ ├── chunk │ │ │ │ ├── QueryAggregation.java │ │ │ │ ├── ReadOnlyChunkInsertionException.java │ │ │ │ ├── Chunk.java │ │ │ │ ├── ChunkImpl.java │ │ │ │ ├── ChunkInfo.java │ │ │ │ ├── OffHeapChunkManagerTask.java │ │ │ │ └── ChunkManager.java │ │ │ │ ├── tagstore │ │ │ │ ├── MatchType.java │ │ │ │ ├── TagStore.java │ │ │ │ ├── Query.java │ │ │ │ ├── Tag.java │ │ │ │ ├── TagMatcher.java │ │ │ │ ├── RoaringBitMapUtils.java │ │ │ │ └── Metric.java │ │ │ │ ├── models │ │ │ │ ├── Points.java │ │ │ │ ├── TimeSeries.java │ │ │ │ └── Point.java │ │ │ │ └── metricandtagstore │ │ │ │ ├── MetricAndTagStore.java │ │ │ │ └── MetricsAndTagStoreImpl.java │ │ └── thrift │ │ │ └── text_message.thrift │ └── test │ │ └── java │ │ └── com │ │ └── pinterest │ │ └── yuvi │ │ ├── tagstore │ │ ├── TagMatcherTest.java │ │ ├── TagTest.java │ │ ├── MetricTest.java │ │ └── QueryTest.java │ │ ├── chunk │ │ ├── MetricUtils.java │ │ └── ChunkImplTest.java │ │ ├── models │ │ └── PointsTest.java │ │ ├── metricstore │ │ ├── VarBitMetricStoreTest.java │ │ └── offheap │ │ │ └── OffHeapVarBitMetricStoreTest.java │ │ ├── metricandtagstore │ │ └── MetricsAndTagStoreImplTest.java │ │ └── bitstream │ │ └── BitStreamTest.java └── pom.xml ├── .travis.yml ├── benchmarks ├── config │ └── log4j.dev.properties ├── src │ └── main │ │ └── java │ │ ├── deltadebug │ │ ├── TestHarness.java │ │ ├── MinimizeTestInput.java │ │ └── DeltaDebug.java │ │ └── com │ │ └── pinterest │ │ └── yuvi │ │ ├── chunk │ │ ├── ChunkManagerStats.java │ │ ├── ChunkManagerIngestorBenchmark.java │ │ ├── ChunkIngestorBenchmark.java │ │ ├── ChunkQueryBenchmark.java │ │ ├── ChunkDataIntegrityChecker.java │ │ ├── ChunkManagerDataIntegrityChecker.java │ │ └── ChunkManagerQueryBenchmark.java │ │ ├── utils │ │ └── MetricUtils.java │ │ ├── metricstore │ │ ├── offheap │ │ │ └── OffHeapVarBitMetricStoreBuildBenchmark.java │ │ └── VarBitMetricStoreBenchmark.java │ │ └── tagstore │ │ └── InvertedIndexTagStoreBenchmark.java ├── README.md └── pom.xml ├── pom.xml ├── README.md └── LICENSE /.gitignore: -------------------------------------------------------------------------------- 1 | .idea/ 2 | *.iml 3 | target/ 4 | *.swp 5 | *~ 6 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/bitstream/ParseException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.bitstream; 2 | 3 | public class ParseException extends RuntimeException { 4 | 5 | public ParseException(String message) { 6 | super(message); 7 | } 8 | } 9 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/writer/MetricWriter.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.writer; 2 | 3 | /** 4 | * MetricWriter interface implements the interfaces for ingesting metrics into Yuvi. 5 | */ 6 | public interface MetricWriter { 7 | 8 | void start(); 9 | 10 | void close(); 11 | } 12 | -------------------------------------------------------------------------------- /yuvi/src/main/thrift/text_message.thrift: -------------------------------------------------------------------------------- 1 | // TextMessage class takes in a list of OpenTSDB metrics as input. 2 | 3 | namespace java com.pinterest.yuvi.thrift 4 | 5 | struct TextMessage { 6 | 1: required list messages; 7 | 2: optional string host; 8 | 3: optional string filename; 9 | } 10 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: java 2 | sudo: required 3 | jdk: 4 | - oraclejdk8 5 | before_install: 6 | - sudo apt-get update -qq 7 | - sudo apt-get install thrift-compiler 8 | - sudo ln -s /usr/bin/thrift /usr/local/bin/thrift 9 | install: 10 | - which thrift 11 | - travis_retry mvn clean install 12 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/metricstore/TimeSeriesIterator.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore; 2 | 3 | import com.pinterest.yuvi.models.Point; 4 | 5 | import java.util.List; 6 | 7 | /** 8 | * An iterator interface for reading the time series data. 9 | */ 10 | public interface TimeSeriesIterator { 11 | 12 | public List getPoints(); 13 | } 14 | -------------------------------------------------------------------------------- /benchmarks/config/log4j.dev.properties: -------------------------------------------------------------------------------- 1 | # log4j logging configuration. 2 | 3 | # root logger. 4 | log4j.rootLogger=INFO, CONSOLE 5 | 6 | log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender 7 | log4j.appender.CONSOLE.Threshold=INFO 8 | log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout 9 | log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] (%F:%L) %-5p %m%n 10 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/chunk/QueryAggregation.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | public enum QueryAggregation { 4 | NONE("none"), 5 | ZIMSUM("zimsum"); 6 | 7 | private final String text; 8 | 9 | private QueryAggregation(final String text) { 10 | this.text = text; 11 | } 12 | 13 | @Override 14 | public String toString() { 15 | return text; 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/chunk/ReadOnlyChunkInsertionException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | /** 4 | * An exception thrown by chunk manager when we try to insert a metric into a read only chunk. 5 | */ 6 | public class ReadOnlyChunkInsertionException extends RuntimeException { 7 | 8 | public ReadOnlyChunkInsertionException(String message) { 9 | super(message); 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/tagstore/MatchType.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | /** 4 | * The match types match all the standard TagVFilter types in OpenTSDB 2.3. More info on these 5 | * types is at: http://opentsdb.net/docs/build/html/user_guide/query/filters.html 6 | */ 7 | public enum MatchType { 8 | EXACT, 9 | WILDCARD, IWILDCARD, 10 | LITERAL_OR, ILITERAL_OR, 11 | NOT_LITERAL_OR, NOT_ILITERAL_OR, 12 | REGEXP 13 | } 14 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 3 | 4.0.0 4 | 5 | com.pinterest 6 | yuvi 7 | 0.1-SNAPSHOT 8 | pom 9 | 10 | 11 | yuvi 12 | benchmarks 13 | 14 | 15 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/deltadebug/TestHarness.java: -------------------------------------------------------------------------------- 1 | package deltadebug; 2 | 3 | import java.util.List; 4 | 5 | /** 6 | * An interface to implement a test harness for evaluating input sets 7 | * for passing or failing/error conditions. 8 | * 9 | * @author Ben Holland 10 | */ 11 | public abstract class TestHarness { 12 | 13 | public static final int PASS = 1; 14 | public static final int UNRESOLVED = 0; 15 | public static final int FAIL = -1; 16 | 17 | /** 18 | * Returns true if the test passes and false if the test fails 19 | * @param 20 | * @param complement 21 | * @return 22 | */ 23 | public abstract int run(List input); 24 | 25 | } 26 | -------------------------------------------------------------------------------- /benchmarks/README.md: -------------------------------------------------------------------------------- 1 | Yuvi Benchmarks 2 | ================ 3 | 4 | Benchmarks for Yuvi store. 5 | 6 | Build the package: 7 | mvn package 8 | 9 | JMH benchmark arguments: 10 | java -jar target/benchmarks.jar com.pinterest.yuvi.chunk.ChunkQueryBenchmark -h 11 | 12 | Running a benchmark with gc profiler: 13 | java -jar target/benchmarks.jar com.pinterest.yuvi.chunk.ChunkQueryBenchmark -jvmArgs -DmetricsData= -prof gc 14 | 15 | Running Chunk data integrity checker: 16 | 17 | java -cp target/benchmarks.jar -DprintErrorKey=true -DmetricsData= -Xmx4G -Xms1G \ 18 | -Dlog4j.configuration=file:///log4j.dev.properties -DtagStoreSize=1000000 \ 19 | -XX:+HeapDumpOnOutOfMemoryError \ 20 | com.pinterest.yuvi.chunk.ChunkDataIntegrityChecker 21 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/writer/kafka/MetricsConsumerRebalanceListener.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.writer.kafka; 2 | 3 | import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; 4 | import org.apache.kafka.common.TopicPartition; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | import java.util.Collection; 9 | 10 | public class MetricsConsumerRebalanceListener implements ConsumerRebalanceListener { 11 | 12 | private static final Logger LOG = LoggerFactory.getLogger(MetricsConsumerRebalanceListener.class); 13 | 14 | MetricsConsumerRebalanceListener() { 15 | LOG.info("Created a metrics rebalance listener."); 16 | } 17 | 18 | @Override 19 | public void onPartitionsRevoked(Collection partitions) { 20 | LOG.info("Revoked partitions {}", partitions); 21 | } 22 | 23 | @Override 24 | public void onPartitionsAssigned(Collection partitions) { 25 | LOG.info("Assigned partitions {}", partitions); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/writer/kafka/MetricsWriterTask.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.writer.kafka; 2 | 3 | import com.pinterest.yuvi.writer.MetricWriter; 4 | 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | public class MetricsWriterTask implements Runnable { 9 | 10 | private final MetricWriter metricWriter; 11 | 12 | private static Logger LOG = LoggerFactory.getLogger(MetricsWriterTask.class); 13 | 14 | public MetricsWriterTask(MetricWriter metricWriter) { 15 | this.metricWriter = metricWriter; 16 | } 17 | 18 | @Override 19 | public void run() { 20 | LOG.info("Starting metrics reader task"); 21 | 22 | // Cleanly shutdown the metrics reader upon exit. 23 | Runtime.getRuntime().addShutdownHook(new Thread() { 24 | @Override 25 | public void run() { 26 | metricWriter.close(); 27 | } 28 | }); 29 | 30 | try { 31 | metricWriter.start(); 32 | } catch (Exception e) { 33 | LOG.error("Error processing metrics", e); 34 | } 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/writer/kafka/ThriftTextMessageDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.writer.kafka; 2 | 3 | 4 | import com.pinterest.yuvi.thrift.TextMessage; 5 | 6 | import org.apache.kafka.common.serialization.Deserializer; 7 | import org.apache.thrift.TDeserializer; 8 | import org.slf4j.Logger; 9 | import org.slf4j.LoggerFactory; 10 | 11 | import java.util.Map; 12 | 13 | public class ThriftTextMessageDeserializer implements Deserializer { 14 | 15 | private static Logger LOG = LoggerFactory.getLogger(ThriftTextMessageDeserializer.class); 16 | private final TDeserializer deserializer = new TDeserializer(); 17 | 18 | @Override 19 | public void configure(Map config, boolean isKey) { 20 | } 21 | 22 | @Override 23 | public TextMessage deserialize(String topic, byte[] data) { 24 | try { 25 | TextMessage textMessage = TextMessage.class.newInstance(); 26 | deserializer.deserialize(textMessage, data); 27 | return textMessage; 28 | } catch (Exception e) { 29 | LOG.error("Failed to parse metrics " + data.toString(), e); 30 | return null; 31 | } 32 | } 33 | 34 | @Override 35 | public void close() { 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/deltadebug/MinimizeTestInput.java: -------------------------------------------------------------------------------- 1 | package deltadebug; 2 | 3 | import com.pinterest.yuvi.chunk.ChunkManagerDataIntegrityChecker; 4 | 5 | import deltadebug.DeltaDebug; 6 | import deltadebug.TestHarness; 7 | 8 | import java.io.IOException; 9 | import java.nio.charset.Charset; 10 | import java.nio.file.Files; 11 | import java.nio.file.Path; 12 | import java.nio.file.Paths; 13 | import java.util.List; 14 | import java.util.stream.Collectors; 15 | 16 | public class MinimizeTestInput { 17 | 18 | private static TestHarness harness = new TestHarness(){ 19 | @Override 20 | public int run(List input) { 21 | // ChunkManagerDataIntegrityChecker checker = new ChunkManagerDataIntegrityChecker(); 22 | // int[] result = checker.checkMetrics(input); 23 | if (2 == 1){ 24 | return FAIL; 25 | } 26 | return PASS; 27 | } 28 | }; 29 | 30 | public static void main(String[] args) throws IOException { 31 | Path path = Paths.get("/tmp/data_input"); 32 | List input = Files.lines(path, Charset.defaultCharset()) 33 | .collect(Collectors.toList()); 34 | 35 | System.out.println(DeltaDebug.ddmin(input, harness)); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/chunk/ChunkManagerStats.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.chunk.Chunk; 4 | import com.pinterest.yuvi.chunk.ChunkManager; 5 | import com.pinterest.yuvi.writer.FileMetricWriter; 6 | 7 | import java.nio.file.Path; 8 | import java.nio.file.Paths; 9 | import java.util.Map; 10 | 11 | public class ChunkManagerStats { 12 | 13 | public static void main(String[] args) { 14 | Path filePath = Paths.get(System.getProperty("metricsData")); 15 | int tagStoreSize = new Integer(System.getProperty("tagStoreSize", "100000")); 16 | String dataDirectoryPath = new String(System.getProperty("dataDirectory", "")); 17 | int moveOffHeap = new Integer(System.getProperty("offHeap", "0")); 18 | 19 | ChunkManager chunkManager = 20 | new ChunkManager("test", tagStoreSize, dataDirectoryPath); 21 | 22 | FileMetricWriter metricWriter = new FileMetricWriter(filePath, chunkManager); 23 | metricWriter.start(); 24 | 25 | if (moveOffHeap != 0) { 26 | chunkManager.toOffHeapChunkMap(); 27 | } 28 | 29 | Map chunkMap = chunkManager.getChunkMap(); 30 | chunkMap.values().forEach(s -> System.out.println(s.getStats())); 31 | } 32 | } 33 | 34 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/metricstore/MetricStore.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore; 2 | 3 | import com.pinterest.yuvi.models.Point; 4 | 5 | import java.util.List; 6 | import java.util.Map; 7 | 8 | /** 9 | * An interface for storing and retrieving individual time-series in-memory. Each time-series is 10 | * identified by a uuid. Also has the ability to serialize itself to an index. This class is 11 | * thread-safe. 12 | */ 13 | public interface MetricStore { 14 | 15 | /** 16 | * Given a id return a list of points contained for that id. 17 | * @param uuid 18 | * @return a list of points. 19 | */ 20 | List getSeries(long uuid); 21 | 22 | 23 | /** 24 | * add a point to an existing time-series, or create a new time-series with the given uuid. 25 | * @param uuid the identifier for the series. 26 | * @param ts unix timestamp in seconds. 27 | * @param val value for the point. 28 | */ 29 | void addPoint(long uuid, long ts, double val); 30 | 31 | Map getStats(); 32 | 33 | Map getSeriesMap(); 34 | 35 | void close(); 36 | 37 | boolean isReadOnly(); 38 | 39 | /** 40 | * Set/unset a store to read only mode. 41 | * @param readOnly 42 | */ 43 | void setReadOnly(boolean readOnly); 44 | } 45 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/models/Points.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.models; 2 | 3 | import java.util.ArrayList; 4 | import java.util.Comparator; 5 | import java.util.Collections; 6 | import java.util.List; 7 | 8 | /** 9 | * Utility classes to work on a set of points. 10 | */ 11 | public class Points { 12 | private Points() { 13 | // private constructor for utility class. 14 | } 15 | 16 | /** 17 | * Sort the points by timestamp and remove data points with duplicate 18 | * timestamps. If two data points have the same timestamp, the second one gets 19 | * precedence. 20 | * @param points a list of points 21 | */ 22 | public static List dedup(List points) { 23 | if (points.isEmpty()) { 24 | return points; 25 | } 26 | 27 | List sortedPoints = new ArrayList<>(points); 28 | // Stable sort 29 | Collections.sort(sortedPoints, Comparator.comparingLong(Point::getTs)); 30 | int bp = 0; 31 | for (int fp = 1; fp < sortedPoints.size(); fp++) { 32 | if (sortedPoints.get(fp).getTs() == sortedPoints.get(bp).getTs()) { 33 | sortedPoints.set(bp, sortedPoints.get(fp)); 34 | } else { 35 | bp++; 36 | sortedPoints.set(bp, sortedPoints.get(fp)); 37 | } 38 | } 39 | return sortedPoints.subList(0, bp + 1); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/tagstore/TagMatcherTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import static org.junit.Assert.assertEquals; 4 | 5 | import junit.framework.TestCase; 6 | import org.junit.Test; 7 | 8 | public class TagMatcherTest { 9 | 10 | private final String testKey = "testKey"; 11 | private final String testValue = "testValue"; 12 | private final Tag testTag = new Tag(testKey, testValue, testKey + "=" + testValue); 13 | 14 | @Test 15 | public void testTagMatcherCreation() { 16 | 17 | TagMatcher m2 = new TagMatcher(MatchType.WILDCARD, testTag); 18 | assertEquals(testTag, m2.tag); 19 | assertEquals(MatchType.WILDCARD, m2.type); 20 | 21 | TagMatcher m4 = TagMatcher.wildcardMatch(testKey, "*"); 22 | assertEquals(new Tag(testKey, "*"), m4.tag); 23 | assertEquals(MatchType.WILDCARD, m4.type); 24 | } 25 | 26 | @Test(expected = IllegalArgumentException.class) 27 | public void testNullType() { 28 | new TagMatcher(null, testTag); 29 | } 30 | 31 | @Test(expected = IllegalArgumentException.class) 32 | public void testNullTag() { 33 | new TagMatcher(MatchType.NOT_ILITERAL_OR, null); 34 | } 35 | 36 | @Test 37 | public void testExactTagMatcher() { 38 | TagMatcher m1 = new TagMatcher(MatchType.EXACT, testTag); 39 | assertEquals(testTag, m1.tag); 40 | assertEquals(MatchType.EXACT, m1.type); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/chunk/MetricUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.tagstore.Metric; 4 | 5 | import java.util.Arrays; 6 | import java.util.List; 7 | 8 | public class MetricUtils { 9 | public static void parseAndAddOpenTSDBMetric(String metricString, Chunk chunk) { 10 | try { 11 | String[] metricParts = metricString.split(" "); 12 | if (metricParts.length > 1 && metricParts[0].equals("put")) { 13 | String metricName = metricParts[1].trim(); 14 | List rawTags = Arrays.asList(metricParts).subList(4, metricParts.length); 15 | Metric metric = new Metric(metricName, rawTags); 16 | long ts = Long.parseLong(metricParts[2].trim()); 17 | double value = Double.parseDouble(metricParts[3].trim()); 18 | 19 | chunk.addPoint(metric, ts, value); 20 | } else { 21 | throw new IllegalArgumentException("Invalid metric string " + metricString); 22 | } 23 | } catch (Exception e) { 24 | throw new IllegalArgumentException("Invalid metric string " + metricString, e); 25 | } 26 | } 27 | 28 | public static String makeMetricString(String metricName, String tags, long ts, double value) { 29 | // Sample msg: put tc.proc.net.compressed.jenkins-worker-mp 1465530393 0 iface=eth0 direction=in 30 | return "put " + metricName + " " + Long.toString(ts) + " " + Double.toString(value) + " " + tags; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/metricandtagstore/MetricAndTagStore.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricandtagstore; 2 | 3 | import com.pinterest.yuvi.models.TimeSeries; 4 | import com.pinterest.yuvi.tagstore.Metric; 5 | import com.pinterest.yuvi.tagstore.Query; 6 | 7 | import java.util.List; 8 | import java.util.Map; 9 | 10 | /** 11 | * This interface provides a higher level interface over the metric and tag store. The interface 12 | * let's users add a metric and a value and similarly retrieve all the points for a given 13 | * and metric. 14 | */ 15 | public interface MetricAndTagStore { 16 | 17 | /** 18 | * Given a id return a list of points contained for that id. 19 | * @param query a Metric query. 20 | * @return a list of points. 21 | */ 22 | List getSeries(Query query); 23 | 24 | /** 25 | * add a point to an existing time-series, or create a new time-series with the given uuid. 26 | * @param metric a metric object. 27 | * @param ts unix timestamp in seconds. 28 | * @param val value for the point. 29 | */ 30 | void addPoint(Metric metric, long ts, double val); 31 | 32 | Map getStats(); 33 | 34 | /** 35 | * Close the metric and tag store cleanly. 36 | */ 37 | void close(); 38 | 39 | /** 40 | * Returns true if the metric or tag store is marked as read only. 41 | */ 42 | boolean isReadOnly(); 43 | 44 | /** 45 | * Enable/disable read only mode for the store. 46 | */ 47 | void setReadOnly(boolean readOnly); 48 | } 49 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Yuvi 2 | 3 | Yuvi is an in-memory storage engine for recent time series metrics data. It has the following features: 4 | 5 | * Implemented in Java. 6 | * Supports OpenTSDB metric ingestion and OpenTSDB queries. 7 | * Uses delta-of-delta encoding from [Facebook Gorilla](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf) to store metrics data. 8 | * Stores tag metadata in an inverted index for fast look ups during queries. 9 | * Stores tag data and older metrics off heap to minimize GC pauses. 10 | * Metrics data older than a few hours is rolled over. 11 | 12 | ## Architecture 13 | Yuvi consists a chunk manager that stores the metrics data the last few hours. 14 | A chunk manager manages several chunk, which consists of 2 hours of metrics data. 15 | Each chunk consists of a _tag store_ to store metrics metdata and a _metric store_ to store the 16 | metrics data. For efficiency, several chunks can also share a tag store. 17 | Once the data in a chunk is older than a configured amount of time, it is removed. 18 | 19 | ## Sample code for using the library 20 | 21 | ```java 22 | import com.pinterest.yuvi.chunk.ChunkManager; 23 | 24 | ChunkManager chunkManager = new ChunkManager("test", 1000); 25 | chunkManager.addMetric("put metricName.cpu.util 1489647603 30 host=host1 cluster=c1"); 26 | List ts = chunkManager.query(Query.parse("metricName.cpu.util host=*"), 1489647600, 1489649600, QueryAggregation.NONE); 27 | ``` 28 | ## NOTE 29 | 30 | This project is under active development. A dev version of OpenTSDB integration code can be found at: https://github.com/mansu/opentsdb/tree/yuvi-dev 31 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/models/TimeSeries.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.models; 2 | 3 | import java.util.List; 4 | 5 | public class TimeSeries { 6 | 7 | private final String metric; 8 | private final List points; 9 | 10 | public TimeSeries(String metric, List points) { 11 | if (metric == null || metric.isEmpty()) { 12 | throw new IllegalArgumentException("Metric name can't be null or empty"); 13 | } 14 | if (points == null) { 15 | throw new IllegalArgumentException("points can't be null"); 16 | } 17 | this.metric = metric; 18 | this.points = points; 19 | } 20 | 21 | @Override 22 | public boolean equals(Object o) { 23 | if (this == o) { 24 | return true; 25 | } 26 | if (o == null || getClass() != o.getClass()) { 27 | return false; 28 | } 29 | 30 | TimeSeries that = (TimeSeries) o; 31 | 32 | if (metric != null ? !metric.equals(that.metric) : that.metric != null) { 33 | return false; 34 | } 35 | return points != null ? points.equals(that.points) : that.points == null; 36 | } 37 | 38 | @Override 39 | public int hashCode() { 40 | int result = metric != null ? metric.hashCode() : 0; 41 | result = 31 * result + (points != null ? points.hashCode() : 0); 42 | return result; 43 | } 44 | 45 | public List getPoints() { 46 | return points; 47 | } 48 | 49 | public String getMetric() { 50 | return metric; 51 | 52 | } 53 | 54 | @Override 55 | public String toString() { 56 | return "TimeSeries{" + "metric='" + metric + '\'' + ", points=" + points + '}'; 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/tagstore/TagTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import static org.junit.Assert.assertEquals; 4 | import static org.junit.Assert.fail; 5 | 6 | import org.junit.Test; 7 | 8 | public class TagTest { 9 | 10 | @Test 11 | public void testTagCreation() { 12 | testInvalidTag(null, "1", "1"); 13 | testInvalidTag("1", null, "1"); 14 | testInvalidTag("1", "1", null); 15 | testInvalidTag("", "1", "1"); 16 | testInvalidTag("1", "", "1"); 17 | testInvalidTag("1", "1", ""); 18 | 19 | Tag t = new Tag("k", "v", "tag"); 20 | assertEquals("k", t.key); 21 | assertEquals("v", t.value); 22 | assertEquals("tag", t.rawTag); 23 | } 24 | 25 | private void testInvalidTag(String key, String value, String tag) { 26 | try { 27 | new Tag(key, value, tag); 28 | fail("Should fail with exception"); 29 | } catch (IllegalArgumentException e) { 30 | } 31 | } 32 | 33 | @Test 34 | public void testTagParse() { 35 | testInvalidTagParse("a"); 36 | testInvalidTagParse("a="); 37 | testInvalidTagParse("=a"); 38 | testInvalidTagParse("="); 39 | 40 | Tag t = Tag.parseTag("k=v"); 41 | assertEquals("k", t.key); 42 | assertEquals("v", t.value); 43 | assertEquals("k=v", t.rawTag); 44 | 45 | Tag t1 = Tag.parseTag("k=v=1"); 46 | assertEquals("k", t1.key); 47 | assertEquals("v=1", t1.value); 48 | assertEquals("k=v=1", t1.rawTag); 49 | } 50 | 51 | private void testInvalidTagParse(String tag) { 52 | try { 53 | Tag.parseTag(tag); 54 | fail("Should fail with exception"); 55 | } catch (IllegalArgumentException e) { 56 | } 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/models/Point.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.models; 2 | 3 | import java.util.ArrayList; 4 | import java.util.Collections; 5 | import java.util.List; 6 | 7 | /** 8 | * An immutable floating-point value at a single point in time. Second-level resolution. 9 | */ 10 | public class Point { 11 | 12 | private final long ts; 13 | private final double val; 14 | 15 | /** 16 | * Create a new point object. 17 | * @param ts unix timestamp in seconds. 18 | * @param val arbitrary value, corresponding to a measurement at the given timestamp. 19 | */ 20 | public Point(long ts, double val) { 21 | this.ts = ts; 22 | this.val = val; 23 | } 24 | 25 | public long getTs() { 26 | return ts; 27 | } 28 | 29 | public double getVal() { 30 | return val; 31 | } 32 | 33 | @Override 34 | public boolean equals(Object o) { 35 | if (this == o) { 36 | return true; 37 | } 38 | if (o == null || getClass() != o.getClass()) { 39 | return false; 40 | } 41 | 42 | Point point = (Point) o; 43 | 44 | if (ts != point.ts) { 45 | return false; 46 | } 47 | if (Double.compare(point.val, val) != 0) { 48 | return false; 49 | } 50 | 51 | return true; 52 | } 53 | 54 | @Override 55 | public int hashCode() { 56 | int result; 57 | long temp; 58 | result = (int) (ts ^ (ts >>> 32)); 59 | temp = Double.doubleToLongBits(val); 60 | result = 31 * result + (int) (temp ^ (temp >>> 32)); 61 | return result; 62 | } 63 | 64 | @Override 65 | public String toString() { 66 | return "Point{" 67 | + "ts=" + ts 68 | + ", val=" + val 69 | + '}'; 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/chunk/Chunk.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.models.TimeSeries; 4 | import com.pinterest.yuvi.tagstore.Metric; 5 | import com.pinterest.yuvi.tagstore.Query; 6 | 7 | import java.util.List; 8 | import java.util.Map; 9 | 10 | /** 11 | * A chunk stores time series data for a specific time range. It can concurrently store metrics and 12 | * respond to queries. Optionally a chunk can be read only at which point it can only be queried. 13 | */ 14 | public interface Chunk { 15 | 16 | /** 17 | * Given a id return a list of points contained for that id. 18 | * @param query a Metric query. 19 | * @return a list of points. 20 | */ 21 | List query(Query query); 22 | 23 | /** 24 | * add a point to an existing time-series, or create a new time-series with the given metric. 25 | * @param metric a metric object. 26 | * @param ts unix timestamp in seconds. 27 | * @param value value for the point. 28 | */ 29 | void addPoint(Metric metric, long ts, double value); 30 | 31 | /** 32 | * A chunk contains some metadata like the list of chunks it can contain. 33 | */ 34 | ChunkInfo info(); 35 | 36 | /** 37 | * A chunk will be available for writes initially. But once we no longer need to write any data to 38 | * it can be turned into a read only chunk. 39 | */ 40 | boolean isReadOnly(); 41 | 42 | /** 43 | * Return true if the chunk contains data within that time range. 44 | */ 45 | boolean containsDataInTimeRange(long startTs, long endTs); 46 | 47 | Map getStats(); 48 | 49 | /** 50 | * Close the chunk. 51 | */ 52 | void close(); 53 | 54 | /** 55 | * Enable/disable read only mode for the store. 56 | */ 57 | void setReadOnly(boolean readOnly); 58 | } 59 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/writer/FileMetricWriter.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.writer; 2 | 3 | import com.pinterest.yuvi.chunk.ChunkManager; 4 | 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | import java.io.IOException; 9 | import java.nio.charset.Charset; 10 | import java.nio.file.Files; 11 | import java.nio.file.Path; 12 | import java.util.stream.Stream; 13 | 14 | /** 15 | * This class reads metrics from a local file and adds them the chunk. This task only ingests the 16 | * metrics from the file once. 17 | */ 18 | public class FileMetricWriter implements MetricWriter { 19 | 20 | private static final Logger LOG = LoggerFactory.getLogger(FileMetricWriter.class); 21 | 22 | final Path metricsFilePath; 23 | 24 | final ChunkManager chunkManager; 25 | 26 | public FileMetricWriter(Path metricsFilePath, ChunkManager chunkManager) { 27 | this.metricsFilePath = metricsFilePath; 28 | this.chunkManager = chunkManager; 29 | } 30 | 31 | @Override 32 | public void start() { 33 | try { 34 | try (Stream lines = Files.lines(metricsFilePath, Charset.defaultCharset())) { 35 | lines.forEachOrdered(line -> { 36 | if (line == null || line.isEmpty()) { // Ignore empty lines 37 | return; 38 | } 39 | try { 40 | chunkManager.addMetric(line); 41 | } catch (Exception e) { 42 | LOG.info("Error ingesting line {} with exception {}", line, e); 43 | } 44 | }); 45 | } 46 | } catch (IOException e) { 47 | LOG.info("Caught exception when ingesting metrics from a file", e); 48 | } 49 | } 50 | 51 | @Override 52 | public void close() { 53 | 54 | } 55 | 56 | public ChunkManager getChunkManager() { 57 | return chunkManager; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/tagstore/TagStore.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import java.util.List; 4 | import java.util.Map; 5 | import java.util.Optional; 6 | 7 | /** 8 | * In time series a single measurement looks as follows: 9 | * cpu.util host=h1 dc=dc1 1234567 10 10 | * metric_name tag1 tag2 timestamp value 11 | * 12 | * A metric is a unique combination of metric_name and the tags. So, 13 | * cpu.util host=h1 dc=dc1 14 | * cpu.util host=h2 dc=dc1 15 | * would be considered 2 different metrics. 16 | * 17 | * In a TSDB, instead of passing the tags around, we would like to refer to them by an integer. 18 | * Further, we would also like to query for metrics by saying get me all metrics with the name 19 | * cpu.util and we should return both the metrics shown above. 20 | * 21 | * A tag store is a storage engine that allows storing and retrieving these tags. So, the API maps 22 | * a metric name and a tag to a UUID and vice versa. 23 | */ 24 | public interface TagStore { 25 | 26 | /** 27 | * For a given metric return an integer id. 28 | * @param metric 29 | * @return 30 | */ 31 | public Optional get(Metric metric); 32 | 33 | /** 34 | * Return an id for the metric if it exists or create a new one and return that. 35 | * @param metric 36 | * @return 37 | */ 38 | public int getOrCreate(Metric metric); 39 | 40 | /** 41 | * Lookup the metric idss that match a given metric query. 42 | * @param metricQuery 43 | * @return 44 | */ 45 | public List lookup(Query metricQuery); 46 | 47 | /** 48 | * Return the name given an ID. 49 | * @param uuid 50 | * @return 51 | */ 52 | String getMetricName(int uuid); 53 | 54 | Map getStats(); 55 | 56 | void close(); 57 | 58 | boolean isReadOnly(); 59 | } 60 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/models/PointsTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.models; 2 | 3 | import junit.framework.TestCase; 4 | import org.hamcrest.collection.IsIterableContainingInOrder; 5 | import org.junit.Assert; 6 | 7 | import java.util.Arrays; 8 | import java.util.Collections; 9 | import java.util.List; 10 | 11 | public class PointsTest extends TestCase { 12 | 13 | public void testDedup() { 14 | Point point1 = new Point(1, 1); 15 | Point point2 = new Point(2, 2); 16 | Point point3 = new Point(3, 3); 17 | 18 | List points = Collections.unmodifiableList(Arrays.asList(point1, point2, point3)); 19 | List 20 | reversedPoints = 21 | Collections.unmodifiableList(Arrays.asList(point3, point2, point1)); 22 | 23 | assertTrue(Points.dedup(Collections.emptyList()).isEmpty()); 24 | Assert.assertThat(points, IsIterableContainingInOrder.contains(Points.dedup(points).toArray())); 25 | Assert.assertThat(points, 26 | IsIterableContainingInOrder.contains(Points.dedup(reversedPoints).toArray())); 27 | 28 | List duplicatePoints = Arrays.asList(point1, new Point(1, 2)); 29 | List result1 = Points.dedup(duplicatePoints); 30 | assertEquals(1, result1.size()); 31 | assertEquals(1, result1.get(0).getTs()); 32 | assertEquals(2.0, result1.get(0).getVal()); 33 | assertEquals(new Point(1, 2), result1.get(0)); 34 | 35 | List duplicatePoints2 = Arrays.asList(point1, new Point(2, 3), point2, point3); 36 | Assert.assertThat(points, 37 | IsIterableContainingInOrder.contains(Points.dedup(duplicatePoints2).toArray())); 38 | 39 | List duplicatePoints3 = Arrays.asList(point1, point2, new Point(2, 3), point2, point3); 40 | Assert.assertThat(points, 41 | IsIterableContainingInOrder.contains(Points.dedup(duplicatePoints3).toArray())); 42 | 43 | List duplicatePoints4 = Arrays.asList(point1, new Point(1, 2), new Point(1, -1)); 44 | List result2 = Points.dedup(duplicatePoints4); 45 | assertEquals(1, result2.size()); 46 | assertEquals(new Point(1, -1), result2.get(0)); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/chunk/ChunkManagerIngestorBenchmark.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.writer.FileMetricWriter; 4 | 5 | import org.openjdk.jmh.annotations.Benchmark; 6 | import org.openjdk.jmh.annotations.BenchmarkMode; 7 | import org.openjdk.jmh.annotations.Fork; 8 | import org.openjdk.jmh.annotations.Level; 9 | import org.openjdk.jmh.annotations.Measurement; 10 | import org.openjdk.jmh.annotations.Mode; 11 | import org.openjdk.jmh.annotations.OutputTimeUnit; 12 | import org.openjdk.jmh.annotations.Scope; 13 | import org.openjdk.jmh.annotations.Setup; 14 | import org.openjdk.jmh.annotations.State; 15 | import org.openjdk.jmh.annotations.Warmup; 16 | import org.openjdk.jmh.infra.Blackhole; 17 | import org.openjdk.jmh.runner.Runner; 18 | import org.openjdk.jmh.runner.RunnerException; 19 | import org.openjdk.jmh.runner.options.Options; 20 | import org.openjdk.jmh.runner.options.OptionsBuilder; 21 | 22 | import java.nio.file.Path; 23 | import java.nio.file.Paths; 24 | import java.util.concurrent.TimeUnit; 25 | 26 | @BenchmarkMode(Mode.AverageTime) 27 | @OutputTimeUnit(TimeUnit.SECONDS) 28 | @Warmup(iterations = 1, time = 1, timeUnit = TimeUnit.SECONDS) 29 | @Measurement(iterations = 3, time = 1, timeUnit = TimeUnit.SECONDS) 30 | @Fork(1) 31 | @State(Scope.Thread) 32 | public class ChunkManagerIngestorBenchmark { 33 | 34 | private ChunkManager chunkManager; 35 | private Path filePath = Paths.get(System.getProperty("metricsData")); 36 | 37 | public static void main(String[] args) throws RunnerException { 38 | Options opt = new OptionsBuilder() 39 | .include(ChunkManagerIngestorBenchmark.class.getSimpleName()) 40 | .build(); 41 | new Runner(opt).run(); 42 | } 43 | 44 | @Setup(Level.Invocation) 45 | public void setup() { 46 | chunkManager = new ChunkManager("test", 1_000_000); 47 | } 48 | 49 | @Benchmark 50 | public void fileIngestTcollectorMetrics(Blackhole bh) throws Exception { 51 | FileMetricWriter metricWriter = new FileMetricWriter(filePath, chunkManager); 52 | metricWriter.start(); 53 | bh.consume(metricWriter); 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/utils/MetricUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.utils; 2 | 3 | import com.pinterest.yuvi.chunk.Chunk; 4 | import com.pinterest.yuvi.tagstore.Metric; 5 | 6 | import java.util.Arrays; 7 | import java.util.Collections; 8 | import java.util.List; 9 | 10 | public class MetricUtils { 11 | public static void parseAndAddOpenTsdbMetric(String metricString, Chunk chunk) { 12 | try { 13 | String[] metricParts = metricString.split(" "); 14 | if (metricParts.length > 1 && metricParts[0].equals("put")) { 15 | String metricName = metricParts[1].trim(); 16 | List rawTags = Arrays.asList(metricParts).subList(4, metricParts.length); 17 | Metric metric = new Metric(metricName, rawTags); 18 | long ts = Long.parseLong(metricParts[2].trim()); 19 | double value = Double.parseDouble(metricParts[3].trim()); 20 | 21 | chunk.addPoint(metric, ts, value); 22 | } else { 23 | throw new IllegalArgumentException("Invalid metric string " + metricString); 24 | } 25 | } catch (Exception e) { 26 | throw new IllegalArgumentException("Invalid metric string " + metricString, e); 27 | } 28 | } 29 | 30 | public static String makeMetricString(String metricName, String tags, long ts, double value) { 31 | // Sample msg: put tc.proc.net.compressed.jenkins-worker-mp 1465530393 0 iface=eth0 direction=in 32 | return "put " + metricName + " " + Long.toString(ts) + " " + Double.toString(value) + " " 33 | + tags; 34 | } 35 | 36 | /** 37 | * Convert a graphite metric to an OpenTSDB metric 38 | */ 39 | public static String graphiteToOpenTsdbMetric(String graphiteMetric) { 40 | String[] words = graphiteMetric.split(" "); 41 | 42 | if (words.length != 3) { 43 | throw new IllegalArgumentException("Invalid graphite metric: " + graphiteMetric); 44 | } 45 | 46 | String metricName = words[0]; 47 | Metric m = new Metric(metricName, Collections.emptyList()); 48 | long ts = Long.parseLong(words[2]); 49 | double value = Double.parseDouble(words[1]); 50 | 51 | return MetricUtils.makeMetricString(metricName, "", ts, value); 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/chunk/ChunkImpl.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.metricandtagstore.MetricAndTagStore; 4 | import com.pinterest.yuvi.models.TimeSeries; 5 | import com.pinterest.yuvi.tagstore.Metric; 6 | import com.pinterest.yuvi.tagstore.Query; 7 | 8 | import org.slf4j.Logger; 9 | import org.slf4j.LoggerFactory; 10 | 11 | import java.util.List; 12 | import java.util.Map; 13 | 14 | public class ChunkImpl implements Chunk { 15 | 16 | private static final Logger LOG = LoggerFactory.getLogger(ChunkImpl.class); 17 | 18 | private final MetricAndTagStore store; 19 | private final ChunkInfo chunkInfo; 20 | 21 | public ChunkImpl(MetricAndTagStore store, ChunkInfo chunkInfo) { 22 | this.store = store; 23 | this.chunkInfo = chunkInfo; 24 | LOG.info("Created a new chunk {}.", chunkInfo); 25 | } 26 | 27 | @Override 28 | public List query(Query query) { 29 | return store.getSeries(query); 30 | } 31 | 32 | @Override 33 | public void addPoint(Metric metric, long ts, double value) { 34 | store.addPoint(metric, ts, value); 35 | } 36 | 37 | @Override 38 | public ChunkInfo info() { 39 | return chunkInfo; 40 | } 41 | 42 | @Override 43 | public boolean isReadOnly() { 44 | return store.isReadOnly(); 45 | } 46 | 47 | @Override 48 | public boolean containsDataInTimeRange(long startTs, long endTs) { 49 | return (chunkInfo.startTimeSecs <= startTs && chunkInfo.endTimeSecs >= startTs) 50 | || (chunkInfo.startTimeSecs <= endTs && chunkInfo.endTimeSecs >= endTs) 51 | || (chunkInfo.startTimeSecs >= startTs && chunkInfo.endTimeSecs <= endTs); 52 | } 53 | 54 | @Override 55 | public Map getStats() { 56 | return store.getStats(); 57 | } 58 | 59 | @Override 60 | public void close() { 61 | store.close(); 62 | } 63 | 64 | @Override 65 | public void setReadOnly(boolean readOnly) { 66 | store.setReadOnly(readOnly); 67 | } 68 | 69 | public MetricAndTagStore getStore() { 70 | return store; 71 | } 72 | 73 | @Override 74 | public String toString() { 75 | return "ChunkImpl{" 76 | + "chunkInfo=" + chunkInfo 77 | + " ,store=" + store 78 | + '}'; 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/chunk/ChunkInfo.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | /** 4 | * This information is exported by each chunk so that the collector is able to schedule queries that 5 | * span across multiple chunks. If a chunk server contains data for a particular data set, the 6 | * server will contain exactly one chunk object that describes which partitions are being served by 7 | * that chunk server. The chunk server will export exactly one ChunkInfo object for that data set. 8 | */ 9 | public class ChunkInfo { 10 | 11 | /** 12 | * A unique identifier for a the data set. The data set corresponds to a logical data source. 13 | * For example, system metrics, or user analytics. The data set may be aggregated by a variety 14 | * of systems, such as kafka and spark. The keyspace of the data set is the set of all metric 15 | * names (including the tags) from that data source. 16 | */ 17 | public final String dataSet; 18 | 19 | public final long startTimeSecs; 20 | 21 | public final long endTimeSecs; 22 | 23 | public ChunkInfo(String dataSet, long startTimeSecs, long endTimeSecs) { 24 | if (dataSet == null || dataSet.isEmpty()) { 25 | throw new IllegalArgumentException("Invalid chunk dataset name " + dataSet); 26 | } 27 | this.dataSet = dataSet; 28 | this.startTimeSecs = startTimeSecs; 29 | this.endTimeSecs = endTimeSecs; 30 | } 31 | 32 | @Override 33 | public String toString() { 34 | return "ChunkInfo{" + 35 | "dataSet='" + dataSet + '\'' + 36 | ", startTimeSecs=" + startTimeSecs + 37 | ", endTimeSecs=" + endTimeSecs + 38 | '}'; 39 | } 40 | 41 | @Override 42 | public boolean equals(Object o) { 43 | if (this == o) { 44 | return true; 45 | } 46 | if (o == null || getClass() != o.getClass()) { 47 | return false; 48 | } 49 | 50 | ChunkInfo chunkInfo = (ChunkInfo) o; 51 | 52 | if (startTimeSecs != chunkInfo.startTimeSecs) { 53 | return false; 54 | } 55 | if (endTimeSecs != chunkInfo.endTimeSecs) { 56 | return false; 57 | } 58 | return dataSet.equals(chunkInfo.dataSet); 59 | } 60 | 61 | @Override 62 | public int hashCode() { 63 | int result = dataSet.hashCode(); 64 | result = 31 * result + (int) (startTimeSecs ^ (startTimeSecs >>> 32)); 65 | result = 31 * result + (int) (endTimeSecs ^ (endTimeSecs >>> 32)); 66 | return result; 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/tagstore/MetricTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import static java.util.Collections.emptyList; 4 | import static junit.framework.TestCase.assertEquals; 5 | import static junit.framework.TestCase.assertTrue; 6 | 7 | import org.junit.Rule; 8 | import org.junit.Test; 9 | import org.junit.rules.ExpectedException; 10 | 11 | import java.util.ArrayList; 12 | import java.util.Arrays; 13 | import java.util.List; 14 | 15 | public class MetricTest { 16 | 17 | @Rule 18 | public ExpectedException thrown = ExpectedException.none(); 19 | 20 | @Test 21 | public void testMetricCreation() { 22 | String metricName = "testMetric"; 23 | Metric m = new Metric(metricName, emptyList()); 24 | assertEquals(metricName, m.metricName); 25 | assertTrue(m.tags.isEmpty()); 26 | assertEquals(metricName, m.fullMetricName); 27 | } 28 | 29 | @Test 30 | public void testMetricWithTagCreation() { 31 | String metricName = "testMetric"; 32 | String tag1 = "k1=v1"; 33 | ArrayList tags = new ArrayList<>(); 34 | tags.add(tag1); 35 | Metric m = new Metric(metricName, tags); 36 | assertEquals(metricName, m.metricName); 37 | assertEquals(1, m.tags.size()); 38 | assertEquals(new Tag("k1", "v1", "k1=v1"), m.tags.get(0)); 39 | assertEquals(Tag.parseTag(tag1), m.tags.get(0)); 40 | assertEquals("testMetric k1=v1", m.fullMetricName); 41 | } 42 | 43 | @Test 44 | public void testMetricWithMultiTagCreation() { 45 | String metricName = "testMetric"; 46 | String tag1 = "k1=v1"; 47 | String tag2 = " k1=v2 "; 48 | String tag3 = " k2=v2"; 49 | String tag4 = "k2=v1 "; 50 | String tag5 = ""; 51 | 52 | List tags = Arrays.asList(tag1, tag2, tag3, tag4, tag5, null); 53 | Metric m = new Metric(metricName, tags); 54 | assertEquals(metricName, m.metricName); 55 | assertEquals(4, m.tags.size()); 56 | assertEquals(new Tag("k1", "v1", "k1=v1"), m.tags.get(0)); 57 | assertEquals(new Tag("k1", "v2", "k1=v2"), m.tags.get(1)); 58 | assertEquals(new Tag("k2", "v2", "k2=v2"), m.tags.get(2)); 59 | assertEquals(new Tag("k2", "v1", "k2=v1"), m.tags.get(3)); 60 | assertEquals("testMetric k1=v1 k1=v2 k2=v1 k2=v2", m.fullMetricName); 61 | } 62 | 63 | @Test 64 | public void testInvalidMetric() throws IllegalArgumentException { 65 | thrown.expect(IllegalArgumentException.class); 66 | new Metric("", emptyList()); 67 | } 68 | 69 | @Test 70 | public void testNullMetricName() throws IllegalArgumentException { 71 | thrown.expect(IllegalArgumentException.class); 72 | new Metric(null, emptyList()); 73 | } 74 | } -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/chunk/ChunkIngestorBenchmark.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.utils.MetricUtils; 4 | import com.pinterest.yuvi.metricandtagstore.MetricsAndTagStoreImpl; 5 | import com.pinterest.yuvi.metricstore.VarBitMetricStore; 6 | import com.pinterest.yuvi.tagstore.InvertedIndexTagStore; 7 | 8 | import org.openjdk.jmh.annotations.Benchmark; 9 | import org.openjdk.jmh.annotations.BenchmarkMode; 10 | import org.openjdk.jmh.annotations.Fork; 11 | import org.openjdk.jmh.annotations.Level; 12 | import org.openjdk.jmh.annotations.Measurement; 13 | import org.openjdk.jmh.annotations.Mode; 14 | import org.openjdk.jmh.annotations.OutputTimeUnit; 15 | import org.openjdk.jmh.annotations.Scope; 16 | import org.openjdk.jmh.annotations.Setup; 17 | import org.openjdk.jmh.annotations.State; 18 | import org.openjdk.jmh.annotations.Warmup; 19 | import org.openjdk.jmh.infra.Blackhole; 20 | import org.openjdk.jmh.runner.Runner; 21 | import org.openjdk.jmh.runner.RunnerException; 22 | import org.openjdk.jmh.runner.options.Options; 23 | import org.openjdk.jmh.runner.options.OptionsBuilder; 24 | 25 | import java.nio.charset.Charset; 26 | import java.nio.file.Files; 27 | import java.nio.file.Path; 28 | import java.nio.file.Paths; 29 | import java.util.concurrent.TimeUnit; 30 | import java.util.stream.Stream; 31 | 32 | @BenchmarkMode(Mode.AverageTime) 33 | @OutputTimeUnit(TimeUnit.SECONDS) 34 | @Warmup(iterations = 2, time = 1, timeUnit = TimeUnit.SECONDS) 35 | @Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.SECONDS) 36 | @Fork(1) 37 | @State(Scope.Thread) 38 | public class ChunkIngestorBenchmark { 39 | 40 | private Chunk chunkStore; 41 | private Path filePath = Paths.get(System.getProperty("metricsData")); 42 | 43 | public static void main(String[] args) throws RunnerException { 44 | Options opt = new OptionsBuilder() 45 | .include(ChunkIngestorBenchmark.class.getSimpleName()) 46 | .build(); 47 | new Runner(opt).run(); 48 | } 49 | 50 | @Setup(Level.Invocation) 51 | public void setup() { 52 | chunkStore = new ChunkImpl( 53 | new MetricsAndTagStoreImpl(new InvertedIndexTagStore(1_000_000, 1_000_000), new VarBitMetricStore()), null); 54 | } 55 | 56 | @Benchmark 57 | public void fileIngestMetrics(Blackhole bh) throws Exception { 58 | try (Stream lines = Files.lines(filePath, Charset.defaultCharset())) { 59 | lines.forEachOrdered(line -> { 60 | try { 61 | MetricUtils.parseAndAddOpenTsdbMetric(line, chunkStore); 62 | } catch (Exception e) { 63 | } 64 | }); 65 | } catch (Exception e) { 66 | e.printStackTrace(); 67 | } 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/deltadebug/DeltaDebug.java: -------------------------------------------------------------------------------- 1 | package deltadebug; 2 | 3 | import java.util.LinkedList; 4 | import java.util.List; 5 | 6 | /** 7 | * A simple Java implementation of Andreas Zeller's Delta Debugging algorithm 8 | * 9 | * Reference: https://www.st.cs.uni-saarland.de/whyprogramsfail/code/dd/DD.java 10 | * 11 | * @author Ben Holland 12 | * 13 | * NOTE: This code is copied from: https://github.com/benjholla/ddmin 14 | */ 15 | public class DeltaDebug { 16 | 17 | /** 18 | * Given an input that causes a failing/error condition, this implementation of a divide and conquer 19 | * algorithm splits the input data into smaller chunks and checks if the smaller input reproduces 20 | * the failing/error condition with a smaller input. 21 | 22 | * @param input The pre-chunked test input to reduce 23 | * @param harness A test harness implementation that returns true (pass) or false (fail/error) for a given input 24 | * 25 | * @return A minimized input that reproduces the failing/error condition 26 | */ 27 | public static List ddmin(List input, TestHarness harness) { 28 | 29 | int n = 2; 30 | while (input.size() >= 2) { 31 | List> subsets = split(input, n); 32 | boolean complementFailing = false; 33 | 34 | for (List subset : subsets){ 35 | List complement = difference(input, subset); 36 | if (harness.run(complement) == TestHarness.FAIL) { 37 | input = complement; 38 | n = Math.max(n - 1, 2); 39 | complementFailing = true; 40 | break; 41 | } 42 | } 43 | 44 | if (!complementFailing) { 45 | if (n == input.size()) { 46 | break; 47 | } 48 | 49 | // increase set granularity 50 | n = Math.min(n * 2, input.size()); 51 | } 52 | } 53 | 54 | return input; 55 | } 56 | 57 | /** 58 | * Returns the difference of set a and set b 59 | * @param a 60 | * @param b 61 | * @return 62 | */ 63 | private static List difference(List a, List b) { 64 | List result = new LinkedList(); 65 | result.addAll(a); 66 | result.removeAll(b); 67 | return result; 68 | } 69 | 70 | /** 71 | * Splits the input set s into n subsets 72 | * @param s 73 | * @param n 74 | * @return 75 | */ 76 | private static List> split(List s, int n) { 77 | List> subsets = new LinkedList>(); 78 | int position = 0; 79 | for (int i = 0; i < n; i++) { 80 | List subset = s.subList(position, position + (s.size() - position) / (n - i)); 81 | subsets.add(subset); 82 | position += subset.size(); 83 | } 84 | return subsets; 85 | } 86 | 87 | } 88 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/metricstore/VarBitMetricStoreTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore; 2 | 3 | import com.pinterest.yuvi.models.Point; 4 | 5 | import junit.framework.TestCase; 6 | import org.junit.Test; 7 | 8 | import java.time.Instant; 9 | import java.util.List; 10 | 11 | public class VarBitMetricStoreTest extends TestCase { 12 | 13 | @Test 14 | public void testEmpty() { 15 | MetricStore store = new VarBitMetricStore(); 16 | assertTrue(store.getSeries(1).isEmpty()); 17 | assertTrue(store.getSeries(2).isEmpty()); 18 | } 19 | 20 | @Test 21 | public void testSimpleInserts() { 22 | MetricStore store = new VarBitMetricStore(); 23 | long uuid1 = 1; 24 | long uuid2 = 2; 25 | long ts = Instant.now().getEpochSecond(); 26 | double value = 100; 27 | 28 | // 1 metric with 1 value. 29 | store.addPoint(uuid1, ts, value); 30 | List points = store.getSeries(uuid1); 31 | assertEquals(1, points.size()); 32 | assertEquals(ts, points.get(0).getTs()); 33 | assertEquals(value, points.get(0).getVal()); 34 | 35 | // 1 metric with 2 values. 36 | store.addPoint(uuid1, ts + 1, value + 1); 37 | List points2 = store.getSeries(uuid1); 38 | assertEquals(2, points2.size()); 39 | assertEquals(ts, points2.get(0).getTs()); 40 | assertEquals(value, points2.get(0).getVal()); 41 | assertEquals(ts + 1, points2.get(1).getTs()); 42 | assertEquals(value + 1, points2.get(1).getVal()); 43 | 44 | // 2 metrics with 2 values each. 45 | store.addPoint(uuid2, ts + 2, value + 2); 46 | List points31 = store.getSeries(uuid1); 47 | assertEquals(2, points31.size()); 48 | assertEquals(ts, points31.get(0).getTs()); 49 | assertEquals(value, points31.get(0).getVal()); 50 | assertEquals(ts + 1, points31.get(1).getTs()); 51 | assertEquals(value + 1, points31.get(1).getVal()); 52 | List points32 = store.getSeries(uuid2); 53 | assertEquals(1, points32.size()); 54 | assertEquals(ts + 2, points32.get(0).getTs()); 55 | assertEquals(value + 2, points32.get(0).getVal()); 56 | 57 | store.addPoint(uuid2, ts + 3, value + 3); 58 | List points41 = store.getSeries(uuid1); 59 | assertEquals(2, points41.size()); 60 | assertEquals(ts, points41.get(0).getTs()); 61 | assertEquals(value, points41.get(0).getVal()); 62 | assertEquals(ts + 1, points41.get(1).getTs()); 63 | assertEquals(value + 1, points41.get(1).getVal()); 64 | List points42 = store.getSeries(uuid2); 65 | assertEquals(2, points42.size()); 66 | assertEquals(ts + 2, points42.get(0).getTs()); 67 | assertEquals(value + 2, points42.get(0).getVal()); 68 | assertEquals(ts + 3, points42.get(1).getTs()); 69 | assertEquals(value + 3, points42.get(1).getVal()); 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/tagstore/QueryTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import static org.junit.Assert.assertEquals; 4 | import static org.junit.Assert.assertTrue; 5 | 6 | import org.junit.Test; 7 | 8 | import java.util.Arrays; 9 | import java.util.Collections; 10 | 11 | public class QueryTest { 12 | private final String metric = "metric"; 13 | private final TagMatcher tagMatcher1 = new TagMatcher(MatchType.EXACT, new Tag("test", "test")); 14 | 15 | @Test 16 | public void testQuery() { 17 | Query q = new Query(metric, Arrays.asList(tagMatcher1)); 18 | assertEquals(metric, q.metricName); 19 | assertEquals(1, q.tagMatchers.size()); 20 | assertEquals(tagMatcher1, q.tagMatchers.get(0)); 21 | 22 | Query q1 = Query.parse(metric); 23 | assertEquals(metric, q1.metricName); 24 | assertTrue(q1.tagMatchers.isEmpty()); 25 | 26 | Query q2 = Query.parse("metric k1=v1"); 27 | assertEquals(metric, q2.metricName); 28 | assertEquals(1, q2.tagMatchers.size()); 29 | assertEquals(new TagMatcher(MatchType.EXACT, new Tag("k1", "v1", "k1=v1")), 30 | q2.tagMatchers.get(0)); 31 | 32 | Query q3 = Query.parse("metric k1=v1 k2=v2"); 33 | assertEquals(metric, q3.metricName); 34 | assertEquals(2, q3.tagMatchers.size()); 35 | assertEquals(new TagMatcher(MatchType.EXACT, new Tag("k1", "v1", "k1=v1")), 36 | q3.tagMatchers.get(0)); 37 | assertEquals(new TagMatcher(MatchType.EXACT, new Tag("k2", "v2", "k2=v2")), 38 | q3.tagMatchers.get(1)); 39 | 40 | Query q4 = Query.parse("metric k1=*"); 41 | assertEquals(metric, q4.metricName); 42 | assertEquals(1, q4.tagMatchers.size()); 43 | assertEquals(new TagMatcher(MatchType.WILDCARD, new Tag("k1", "*")), 44 | q4.tagMatchers.get(0)); 45 | 46 | Query q5 = Query.parse("metric k1=* k2=v1"); 47 | assertEquals(metric, q5.metricName); 48 | assertEquals(2, q5.tagMatchers.size()); 49 | assertEquals(new TagMatcher(MatchType.WILDCARD, new Tag("k1", "*")), 50 | q5.tagMatchers.get(0)); 51 | assertEquals(new TagMatcher(MatchType.EXACT, new Tag("k2", "v1", "k2=v1")), 52 | q5.tagMatchers.get(1)); 53 | } 54 | 55 | @Test(expected = IllegalArgumentException.class) 56 | public void testNullMetricName() { 57 | new Query(null, Collections.emptyList()); 58 | } 59 | 60 | @Test(expected = IllegalArgumentException.class) 61 | public void testEmptyMetricName() { 62 | new Query(null, Collections.emptyList()); 63 | } 64 | 65 | @Test(expected = IllegalArgumentException.class) 66 | public void testNullTags() { 67 | new Query(metric, null); 68 | } 69 | 70 | @Test(expected = IllegalArgumentException.class) 71 | public void testDuplicateTagKeysUsingConstructor() { 72 | new Query(metric, Arrays.asList(tagMatcher1, tagMatcher1)); 73 | } 74 | 75 | @Test(expected = IllegalArgumentException.class) 76 | public void testDuplicateTagKeysUsingParse() { 77 | Query.parse("metric k1=v1 k1=v2"); 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/metricandtagstore/MetricsAndTagStoreImpl.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricandtagstore; 2 | 3 | import com.pinterest.yuvi.metricstore.MetricStore; 4 | import com.pinterest.yuvi.models.TimeSeries; 5 | import com.pinterest.yuvi.tagstore.Metric; 6 | import com.pinterest.yuvi.tagstore.Query; 7 | import com.pinterest.yuvi.tagstore.TagStore; 8 | 9 | import org.slf4j.Logger; 10 | import org.slf4j.LoggerFactory; 11 | 12 | import java.util.HashMap; 13 | import java.util.List; 14 | import java.util.Map; 15 | import java.util.stream.Collectors; 16 | 17 | /** 18 | * MetricAndTagStore stores the metrics in gorilla format and the tags in an inverted index. 19 | * 20 | * TODO: Batch insert API 21 | * TODO: synchronization? 22 | * TODO: error handling 23 | */ 24 | public class MetricsAndTagStoreImpl implements MetricAndTagStore { 25 | 26 | private static final Logger LOG = LoggerFactory.getLogger(MetricsAndTagStoreImpl.class); 27 | private final TagStore tagStore; 28 | private final MetricStore metricStore; 29 | 30 | public MetricsAndTagStoreImpl(TagStore tagStore, MetricStore metricStore) { 31 | this.tagStore = tagStore; 32 | this.metricStore = metricStore; 33 | LOG.info("Created a new metric store {} and tag store {}.", metricStore, tagStore); 34 | } 35 | 36 | @Override 37 | public List getSeries(Query query) { 38 | List ids = tagStore.lookup(query); 39 | // Get an iterator instead? 40 | // Catch exceptions and make it easy to debug. 41 | return ids.stream() 42 | .map(id -> new TimeSeries(tagStore.getMetricName(id), metricStore.getSeries(id))) 43 | .collect(Collectors.toList()); 44 | } 45 | 46 | public MetricStore getMetricStore() { 47 | return metricStore; 48 | } 49 | 50 | public TagStore getTagStore() { 51 | return tagStore; 52 | } 53 | 54 | @Override 55 | public void addPoint(Metric metric, long ts, double val) { 56 | int metricId = tagStore.getOrCreate(metric); 57 | metricStore.addPoint(metricId, ts, val); 58 | 59 | } 60 | 61 | @Override 62 | public Map getStats() { 63 | Map stats = new HashMap<>(); 64 | metricStore.getStats().entrySet() 65 | .forEach(entry -> stats.put("metricStore_" + entry.getKey(), entry.getValue())); 66 | tagStore.getStats().entrySet() 67 | .forEach(entry -> stats.put("tagStore_" + entry.getKey(), entry.getValue())); 68 | return stats; 69 | } 70 | 71 | /** 72 | * Currently, tag store is shared among chunks. So only close the metric store. 73 | */ 74 | @Override 75 | public void close() { 76 | metricStore.close(); 77 | } 78 | 79 | @Override 80 | public boolean isReadOnly() { 81 | return tagStore.isReadOnly() || metricStore.isReadOnly(); 82 | } 83 | 84 | @Override 85 | public void setReadOnly(boolean readOnly) { 86 | // Currently, a tag store can't be set to read only since it is shard by all chunks. 87 | metricStore.setReadOnly(readOnly); 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/tagstore/Query.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import static java.util.stream.Collectors.groupingBy; 4 | import static java.util.stream.Collectors.mapping; 5 | import static java.util.stream.Collectors.toList; 6 | import static java.util.Map.Entry; 7 | 8 | import java.util.AbstractMap.SimpleEntry; 9 | import java.util.ArrayList; 10 | import java.util.Arrays; 11 | import java.util.List; 12 | import java.util.Map; 13 | 14 | /** 15 | * A Query contains instructions to fetch a set of time series associated with a single metric 16 | * name. All time series have the given metric name and match the given set of tag matchers 17 | * returned by the query. Since the tag matcher query resolution is complex, we only allow one 18 | * tagMatcher per tagKey to keep the query resolution deterministic. This is an OpenTSDB limitation. 19 | */ 20 | public final class Query { 21 | 22 | public final String metricName; 23 | public final List tagMatchers; 24 | 25 | public Query(final String metricName, final List tagMatchers) { 26 | if (metricName == null || metricName.isEmpty() || tagMatchers == null) { 27 | throw new IllegalArgumentException("metric name or tag matcher can't be null."); 28 | } 29 | 30 | final Map> tagNameMap = tagMatchers.stream() 31 | .map(t -> new SimpleEntry<>(t.tag.key, t)) 32 | .collect(groupingBy(Entry::getKey, mapping(Entry::getValue, toList()))); 33 | 34 | tagNameMap.entrySet().forEach(tagKeyEntry -> { 35 | if (tagKeyEntry.getValue().size() != 1) { 36 | throw new IllegalArgumentException("Only one tagFilter is allowed per tagKey: " 37 | + tagKeyEntry.getKey() + " .But we found " + tagKeyEntry.getValue().toString()); 38 | } 39 | }); 40 | 41 | this.metricName = metricName; 42 | this.tagMatchers = tagMatchers; 43 | } 44 | 45 | /** 46 | * Parse a string into a query. This should only be used to make unit tests more concise. 47 | * Production code should take a more rigorous approach to query parsing 48 | * @param s a string of the form "metric.name dimension1=label1 dimension2=* ..." 49 | * @return a new Metric 50 | * @throws Exception if parsing failed 51 | */ 52 | public static Query parse(String s) { 53 | List splits = Arrays.asList(s.split(" ")); 54 | String metricName = splits.get(0); 55 | List matchers = new ArrayList<>(); 56 | for (String s2 : splits.subList(1, splits.size())) { 57 | Tag tag = Tag.parseTag(s2); 58 | if (tag.value.equals("*")) { 59 | matchers.add(TagMatcher.wildcardMatch(tag.key, "*")); 60 | } else { 61 | matchers.add(TagMatcher.exactMatch(tag)); 62 | } 63 | } 64 | return new Query(metricName, matchers); 65 | } 66 | 67 | @Override 68 | public String toString() { 69 | return "Query{" + 70 | "metricName='" + metricName + " " + 71 | ", tagMatchers=" + tagMatchers + 72 | '}'; 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/tagstore/Tag.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import static com.pinterest.yuvi.tagstore.Metric.TAG_DELIMITER; 4 | 5 | import org.apache.commons.lang3.StringUtils; 6 | 7 | 8 | /** 9 | * A tag is a key=value pair. The key and value must be ASCII. 10 | */ 11 | public final class Tag implements Comparable { 12 | 13 | private static final String EMPTY_RAW_TAG = "="; 14 | 15 | public final String rawTag; 16 | public final String key; 17 | public final String value; 18 | 19 | public Tag(String key, String value, String rawTag) { 20 | if (StringUtils.isNotBlank(key) && StringUtils.isNotBlank(value) 21 | && StringUtils.isNotBlank(rawTag)) { 22 | this.key = key; 23 | this.value = value; 24 | this.rawTag = rawTag; 25 | } else { 26 | throw new IllegalArgumentException("Input values can't be blank : " 27 | + key + " " + value + " " + rawTag); 28 | } 29 | } 30 | 31 | public Tag(String key, String value) { 32 | this(key, value, EMPTY_RAW_TAG); 33 | } 34 | 35 | @Override 36 | public int compareTo(Tag o) { 37 | int k = key.compareTo(o.key); 38 | if (k != 0) { 39 | return k; 40 | } 41 | return value.compareTo(o.value); 42 | } 43 | 44 | @Override 45 | public boolean equals(Object o) { 46 | if (this == o) { 47 | return true; 48 | } 49 | if (o == null || getClass() != o.getClass()) { 50 | return false; 51 | } 52 | 53 | Tag tag = (Tag) o; 54 | 55 | if (rawTag != null ? !rawTag.equals(tag.rawTag) : tag.rawTag != null) { 56 | return false; 57 | } 58 | if (key != null ? !key.equals(tag.key) : tag.key != null) { 59 | return false; 60 | } 61 | return value != null ? value.equals(tag.value) : tag.value == null; 62 | 63 | } 64 | 65 | @Override 66 | public int hashCode() { 67 | int result = rawTag != null ? rawTag.hashCode() : 0; 68 | result = 31 * result + (key != null ? key.hashCode() : 0); 69 | result = 31 * result + (value != null ? value.hashCode() : 0); 70 | return result; 71 | } 72 | 73 | /** 74 | * This function parses a key=value tag into this object. Each tag is a parsed using indexOf 75 | * instead of String.split for optimal performance. This code is called to parse every metric 76 | * ingested and on every query. So, it is very imporant to keep this code fast. 77 | * 78 | * @param rawTag 79 | * @return parsed Tag object 80 | */ 81 | public static Tag parseTag(String rawTag) { 82 | int index = getDelimiterIndex(rawTag); 83 | String key = rawTag.substring(0, index); 84 | String value = rawTag.substring(index + 1); 85 | if (key.isEmpty() || value.isEmpty()) { 86 | throw new IllegalArgumentException("Invalid rawTag" + rawTag); 87 | } 88 | return new Tag(key, value, rawTag); 89 | } 90 | 91 | private static int getDelimiterIndex(String rawTag) { 92 | // Don't use string or split here since it has lower perf. 93 | int index = rawTag.indexOf(TAG_DELIMITER); 94 | if (index == -1) { 95 | throw new IllegalArgumentException("Invalid rawTag " + rawTag); 96 | } 97 | return index; 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/metricstore/offheap/OffHeapVarBitMetricStoreBuildBenchmark.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore.offheap; 2 | 3 | import com.pinterest.yuvi.metricandtagstore.MetricsAndTagStoreImpl; 4 | import com.pinterest.yuvi.metricstore.VarBitMetricStore; 5 | import com.pinterest.yuvi.utils.MetricUtils; 6 | import com.pinterest.yuvi.chunk.Chunk; 7 | import com.pinterest.yuvi.chunk.ChunkImpl; 8 | import com.pinterest.yuvi.tagstore.InvertedIndexTagStore; 9 | 10 | import org.openjdk.jmh.annotations.Benchmark; 11 | import org.openjdk.jmh.annotations.BenchmarkMode; 12 | import org.openjdk.jmh.annotations.Fork; 13 | import org.openjdk.jmh.annotations.Level; 14 | import org.openjdk.jmh.annotations.Measurement; 15 | import org.openjdk.jmh.annotations.Mode; 16 | import org.openjdk.jmh.annotations.OutputTimeUnit; 17 | import org.openjdk.jmh.annotations.Scope; 18 | import org.openjdk.jmh.annotations.Setup; 19 | import org.openjdk.jmh.annotations.State; 20 | import org.openjdk.jmh.annotations.Warmup; 21 | import org.openjdk.jmh.infra.Blackhole; 22 | import org.openjdk.jmh.runner.Runner; 23 | import org.openjdk.jmh.runner.RunnerException; 24 | import org.openjdk.jmh.runner.options.Options; 25 | import org.openjdk.jmh.runner.options.OptionsBuilder; 26 | 27 | import java.nio.charset.Charset; 28 | import java.nio.file.Files; 29 | import java.nio.file.Path; 30 | import java.nio.file.Paths; 31 | import java.util.HashMap; 32 | import java.util.Map; 33 | import java.util.concurrent.TimeUnit; 34 | import java.util.stream.Stream; 35 | 36 | @BenchmarkMode(Mode.AverageTime) 37 | @OutputTimeUnit(TimeUnit.SECONDS) 38 | @Warmup(iterations = 2, time = 1, timeUnit = TimeUnit.MILLISECONDS) 39 | @Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.MILLISECONDS) 40 | @Fork(1) 41 | @State(Scope.Thread) 42 | public class OffHeapVarBitMetricStoreBuildBenchmark { 43 | 44 | private Chunk chunkStore; 45 | private Path filePath = Paths.get(System.getProperty("metricsData")); 46 | private HashMap counts = new HashMap(); 47 | 48 | public static void main(String[] args) throws RunnerException { 49 | Options opt = new OptionsBuilder() 50 | .include(OffHeapVarBitMetricStoreBuildBenchmark.class.getSimpleName()) 51 | .build(); 52 | new Runner(opt).run(); 53 | } 54 | 55 | @Setup(Level.Trial) 56 | public void setup() { 57 | chunkStore = new ChunkImpl( 58 | new MetricsAndTagStoreImpl(new InvertedIndexTagStore(1_000_000, 1_000_000), new VarBitMetricStore()), null); 59 | 60 | try (Stream lines = Files.lines(filePath, Charset.defaultCharset())) { 61 | lines.forEachOrdered(line -> { 62 | try { 63 | String[] words = line.split(" "); 64 | String metricName = words[1]; 65 | if (counts.containsKey(metricName)) { 66 | counts.put(metricName, counts.get(metricName) + 1); 67 | } else { 68 | counts.put(metricName, 1); 69 | } 70 | 71 | MetricUtils.parseAndAddOpenTsdbMetric(line, chunkStore); 72 | } catch (Exception e) { 73 | } 74 | }); 75 | } catch (Exception e) { 76 | e.printStackTrace(); 77 | } 78 | } 79 | 80 | @Benchmark 81 | public void creationTime(Blackhole bh) { 82 | Map seriesMap = ((MetricsAndTagStoreImpl) ((ChunkImpl) chunkStore).getStore()).getMetricStore() 83 | .getSeriesMap(); 84 | OffHeapVarBitMetricStore newStore = OffHeapVarBitMetricStore.toOffHeapStore(seriesMap, "", ""); 85 | bh.consume(newStore); 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/chunk/ChunkQueryBenchmark.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.utils.MetricUtils; 4 | import com.pinterest.yuvi.metricandtagstore.MetricsAndTagStoreImpl; 5 | import com.pinterest.yuvi.metricstore.VarBitMetricStore; 6 | import com.pinterest.yuvi.models.TimeSeries; 7 | import com.pinterest.yuvi.tagstore.InvertedIndexTagStore; 8 | import com.pinterest.yuvi.tagstore.Query; 9 | 10 | import org.openjdk.jmh.annotations.Benchmark; 11 | import org.openjdk.jmh.annotations.BenchmarkMode; 12 | import org.openjdk.jmh.annotations.Fork; 13 | import org.openjdk.jmh.annotations.Level; 14 | import org.openjdk.jmh.annotations.Measurement; 15 | import org.openjdk.jmh.annotations.Mode; 16 | import org.openjdk.jmh.annotations.OutputTimeUnit; 17 | import org.openjdk.jmh.annotations.Scope; 18 | import org.openjdk.jmh.annotations.Setup; 19 | import org.openjdk.jmh.annotations.State; 20 | import org.openjdk.jmh.annotations.Warmup; 21 | import org.openjdk.jmh.infra.Blackhole; 22 | import org.openjdk.jmh.runner.Runner; 23 | import org.openjdk.jmh.runner.RunnerException; 24 | import org.openjdk.jmh.runner.options.Options; 25 | import org.openjdk.jmh.runner.options.OptionsBuilder; 26 | 27 | import java.nio.charset.Charset; 28 | import java.nio.file.Files; 29 | import java.nio.file.Path; 30 | import java.nio.file.Paths; 31 | import java.util.HashMap; 32 | import java.util.List; 33 | import java.util.concurrent.TimeUnit; 34 | import java.util.stream.Stream; 35 | 36 | @BenchmarkMode(Mode.AverageTime) 37 | @OutputTimeUnit(TimeUnit.SECONDS) 38 | @Warmup(iterations = 1, time = 1, timeUnit = TimeUnit.MILLISECONDS) 39 | @Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.MILLISECONDS) 40 | @Fork(1) 41 | @State(Scope.Thread) 42 | public class ChunkQueryBenchmark { 43 | 44 | private Chunk chunkStore; 45 | private Path filePath = Paths.get(System.getProperty("metricsData")); 46 | 47 | private HashMap counts = new HashMap(); 48 | 49 | public static void main(String[] args) throws RunnerException { 50 | Options opt = new OptionsBuilder() 51 | .include(ChunkQueryBenchmark.class.getSimpleName()) 52 | .build(); 53 | new Runner(opt).run(); 54 | } 55 | 56 | @Setup(Level.Trial) 57 | public void setup() { 58 | chunkStore = new ChunkImpl( 59 | new MetricsAndTagStoreImpl(new InvertedIndexTagStore(1_000_000, 1_000_000), new VarBitMetricStore()), 60 | null); 61 | 62 | try (Stream lines = Files.lines(filePath, Charset.defaultCharset())) { 63 | lines.forEachOrdered(line -> { 64 | try { 65 | String[] words = line.split(" "); 66 | String metricName = words[1]; 67 | if (metricName != null && !metricName.isEmpty()) { 68 | if (counts.containsKey(metricName)) { 69 | counts.put(metricName, counts.get(metricName) + 1); 70 | } else { 71 | counts.put(metricName, 1); 72 | } 73 | 74 | MetricUtils.parseAndAddOpenTsdbMetric(line, chunkStore); 75 | } 76 | } catch (Exception e) { 77 | System.err.println("Error ingesting metric: " + e.getMessage()); 78 | e.printStackTrace(); 79 | } 80 | }); 81 | } catch (Exception e) { 82 | e.printStackTrace(); 83 | } 84 | } 85 | 86 | @Benchmark 87 | public void queryAllMetrics(Blackhole bh) { 88 | counts.keySet().stream().forEach(k -> { 89 | List points = chunkStore.query(Query.parse(k)); 90 | bh.consume(points.size()); 91 | }); 92 | } 93 | } 94 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/bitstream/BitStreamIterator.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.bitstream; 2 | 3 | import com.google.common.annotations.VisibleForTesting; 4 | 5 | import java.util.Arrays; 6 | 7 | /** 8 | * Provides functions for reading a sequence of raw bits from a binary array. Bits are read 9 | * sequentially, and are addressed by their offset from the beginning of the array. Not thread 10 | * safe. 11 | * 12 | * TODO: Take the BitStream object as input for this class. 13 | * TODO: Implement Java Iterator interface. 14 | * TODO: Investigate if is long array is the best or if a int array if more performant. 15 | */ 16 | public class BitStreamIterator { 17 | 18 | private final long[] data; 19 | private final int bitLength; 20 | private int idx; 21 | private byte shift; 22 | 23 | /** 24 | * Construct a reader that starts reading from the beginning of the array. 25 | * @param data the raw binary data, big-endian 26 | * @param bitLength the number of bits in the array. Attempts to access bits at offset > 27 | * bitLength will 28 | * result in a ParseException. 29 | */ 30 | BitStreamIterator(long[] data, int bitLength) { 31 | this.data = data; 32 | this.bitLength = bitLength; 33 | } 34 | 35 | /** 36 | * Get the offset of the next bit to read. 37 | * @return the offset from the beginning of the array. 38 | */ 39 | int bitOffset() { 40 | return idx * 64 + shift; 41 | } 42 | 43 | /** 44 | * Read up to 64 consecutive bits from the array. 45 | * @param n number of bits to read, between 0 and 64. 46 | * @return a long integer containing the bits in the n least-significant positions. 47 | * @throws ParseException if the reader has reached the end of the bit array. 48 | */ 49 | public long read(int n) { 50 | if (n < 0 && n > 64) { 51 | throw new IllegalArgumentException(n + " should be less than 64 bits."); 52 | } 53 | 54 | int want = bitOffset() + n; 55 | if (want > bitLength) { 56 | throw new ParseException("Out of bounds: bitLength=" + bitLength + " want=" + want); 57 | } 58 | 59 | long result; 60 | if (64 - shift > n) { 61 | result = data[idx] << shift >>> 64 - n; 62 | shift += n; 63 | } else { 64 | result = data[idx] << shift >>> shift; 65 | shift += n; 66 | if (shift >= 64) { 67 | shift -= 64; 68 | idx++; 69 | } 70 | if (shift != 0) { 71 | result = (result << shift) | (data[idx] >>> 64 - shift); 72 | } 73 | } 74 | return result; 75 | } 76 | 77 | /** 78 | * Peek at the next n bits, and consume them if they match val. 79 | * @param n the number of bits to read 80 | * @param val the bits to compare against. 81 | * @return True if the bits match, and the cursor advances. 82 | * @throws ParseException if the reader has reached the end of the bit array. 83 | */ 84 | public boolean tryRead(int n, long val) throws ParseException { 85 | long v = read(n); 86 | if (val == v) { 87 | return true; 88 | } 89 | shift -= n; 90 | if (shift < 0) { 91 | shift += 64; 92 | idx--; 93 | } 94 | return false; 95 | } 96 | 97 | @Override 98 | public String toString() { 99 | return "BitStreamIterator{" 100 | + "data=" + Arrays.toString(data) 101 | + ", bitLength=" + bitLength 102 | + ", idx=" + idx 103 | + ", shift=" + shift 104 | + '}'; 105 | } 106 | 107 | @VisibleForTesting 108 | public int getBitLength() { 109 | return bitLength; 110 | } 111 | } 112 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/tagstore/InvertedIndexTagStoreBenchmark.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import org.openjdk.jmh.annotations.Benchmark; 4 | import org.openjdk.jmh.annotations.BenchmarkMode; 5 | import org.openjdk.jmh.annotations.Fork; 6 | import org.openjdk.jmh.annotations.Level; 7 | import org.openjdk.jmh.annotations.Measurement; 8 | import org.openjdk.jmh.annotations.Mode; 9 | import org.openjdk.jmh.annotations.OutputTimeUnit; 10 | import org.openjdk.jmh.annotations.Param; 11 | import org.openjdk.jmh.annotations.Scope; 12 | import org.openjdk.jmh.annotations.Setup; 13 | import org.openjdk.jmh.annotations.State; 14 | import org.openjdk.jmh.annotations.Warmup; 15 | import org.openjdk.jmh.infra.Blackhole; 16 | import org.openjdk.jmh.runner.Runner; 17 | import org.openjdk.jmh.runner.RunnerException; 18 | import org.openjdk.jmh.runner.options.Options; 19 | import org.openjdk.jmh.runner.options.OptionsBuilder; 20 | 21 | import java.util.ArrayList; 22 | import java.util.List; 23 | import java.util.concurrent.ThreadLocalRandom; 24 | import java.util.concurrent.TimeUnit; 25 | 26 | @BenchmarkMode(Mode.AverageTime) 27 | @OutputTimeUnit(TimeUnit.MILLISECONDS) 28 | @Warmup(iterations = 2, time = 1, timeUnit = TimeUnit.SECONDS) 29 | @Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.SECONDS) 30 | @Fork(1) 31 | @State(Scope.Thread) 32 | public class InvertedIndexTagStoreBenchmark { 33 | 34 | private static final int numMetrics = 100; 35 | private static final int numKeys = 4; 36 | private static final int numValues = 4; 37 | 38 | private List metrics; 39 | 40 | @Param({"InvertedIndexTagStore"}) 41 | public String msType; 42 | 43 | private InvertedIndexTagStore ms; 44 | 45 | public static Metric randomMetric(int numMetrics, int numDimensions, int numLabels) { 46 | ThreadLocalRandom r = ThreadLocalRandom.current(); 47 | int i = r.nextInt(0, numMetrics); 48 | int[] labs = new int[numDimensions]; 49 | for (int j = 0; j < numDimensions; j++) { 50 | labs[j] = r.nextInt(0, numLabels); 51 | } 52 | return metric(i, labs); 53 | } 54 | 55 | private static String metricName(int i) { 56 | return "the.metric.name.is.very.very.very.very.very.long" + i; 57 | } 58 | 59 | private static Metric metric(int i, int[] labs) { 60 | List tags = new ArrayList(); 61 | for (int j = 0; j < labs.length; j++) { 62 | tags.add("key" + j + "=" + "value" + labs[j]); 63 | } 64 | return new Metric(metricName(i), tags); 65 | } 66 | 67 | public static void main(String[] args) throws RunnerException { 68 | Options opt = new OptionsBuilder() 69 | .include(InvertedIndexTagStoreBenchmark.class.getSimpleName()) 70 | .build(); 71 | 72 | new Runner(opt).run(); 73 | } 74 | 75 | @Setup(Level.Invocation) 76 | public void setup() { 77 | switch (msType) { 78 | case "InvertedIndexTagStore": 79 | int initialMapSize = 10000; 80 | ms = new InvertedIndexTagStore(initialMapSize, initialMapSize); 81 | metrics = new ArrayList(); 82 | for (int i = 0; i < initialMapSize; i++) { 83 | metrics.add(randomMetric(numMetrics, numKeys, numValues)); 84 | } 85 | break; 86 | default: 87 | throw new RuntimeException("invalid msType: " + msType); 88 | } 89 | } 90 | 91 | @Benchmark 92 | public void insertMetricThenFetch(Blackhole bh) { 93 | for (Metric m : metrics) { 94 | ms.getOrCreate(m); 95 | bh.consume(ms.get(m)); 96 | } 97 | } 98 | 99 | // TODO: memory used by roaring bitmap. Where is it? 100 | // TODO: Time taken to answer queries as a function of metric size and metric type. 101 | } 102 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/chunk/ChunkDataIntegrityChecker.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.utils.MetricUtils; 4 | import com.pinterest.yuvi.metricandtagstore.MetricsAndTagStoreImpl; 5 | import com.pinterest.yuvi.metricstore.VarBitMetricStore; 6 | import com.pinterest.yuvi.models.TimeSeries; 7 | import com.pinterest.yuvi.tagstore.InvertedIndexTagStore; 8 | import com.pinterest.yuvi.tagstore.Query; 9 | 10 | import java.io.IOException; 11 | import java.nio.charset.Charset; 12 | import java.nio.file.Files; 13 | import java.nio.file.Path; 14 | import java.nio.file.Paths; 15 | import java.util.HashMap; 16 | import java.util.List; 17 | import java.util.stream.Stream; 18 | 19 | /** 20 | * This program takes a raw metrics file as input, ingests the data in it into a Yuvi Chunk, 21 | * reads it back and compares it to input data. 22 | * 23 | * For supporting large files, it compares the number of data points per metric instead of 24 | * actual data points. 25 | */ 26 | public class ChunkDataIntegrityChecker { 27 | 28 | private final Path filePath; 29 | private final int tagStoreSize; 30 | 31 | public ChunkDataIntegrityChecker() { 32 | this(System.getProperty("metricsData"), System.getProperty("tagStoreSize", "1000000")); 33 | } 34 | 35 | public ChunkDataIntegrityChecker(String filePathName, String tagStoreSize) { 36 | if (filePathName == null || filePathName.isEmpty()) { 37 | throw new IllegalArgumentException("filePathName can't be empty"); 38 | } 39 | 40 | filePath = Paths.get(filePathName); 41 | this.tagStoreSize = new Integer(tagStoreSize); 42 | } 43 | 44 | public static void main(String[] args) throws IOException { 45 | ChunkDataIntegrityChecker checker = new ChunkDataIntegrityChecker(); 46 | checker.checkData(); 47 | } 48 | 49 | void checkData() throws IOException { 50 | Chunk chunkStore = new ChunkImpl( 51 | new MetricsAndTagStoreImpl(new InvertedIndexTagStore(tagStoreSize, tagStoreSize), new VarBitMetricStore()), 52 | null); 53 | 54 | HashMap counts = new HashMap(); 55 | 56 | try (Stream lines = Files.lines(filePath, Charset.defaultCharset())) { 57 | lines.forEachOrdered(line -> { 58 | if (line == null || line.isEmpty()) { 59 | return; 60 | } 61 | 62 | try { 63 | String[] words = line.split(" "); 64 | String metricName = words[1]; 65 | if (metricName != null && !metricName.isEmpty()) { 66 | if (counts.containsKey(metricName)) { 67 | counts.put(metricName, counts.get(metricName) + 1); 68 | } else { 69 | counts.put(metricName, 1); 70 | } 71 | 72 | MetricUtils.parseAndAddOpenTsdbMetric(line, chunkStore); 73 | } else { 74 | System.out.println(metricName + " " + line); 75 | } 76 | } catch (Exception e) { 77 | System.out.println("error: " + e.getMessage()); 78 | e.printStackTrace(); 79 | } 80 | }); 81 | } 82 | 83 | final int[] stats = new int[3]; 84 | stats[0] = 0; 85 | stats[1] = 0; 86 | counts.keySet().stream().forEach(k -> { 87 | try { 88 | List timeSeriesList = chunkStore.query(Query.parse(k)); 89 | int pointsCount = timeSeriesList.stream().mapToInt(l -> l.getPoints().size()).sum(); 90 | if (counts.get(k) != pointsCount) { 91 | stats[2] = stats[2] + 1; 92 | } 93 | stats[1] = stats[1] + 1; 94 | } catch (Exception e) { 95 | stats[0] = stats[0] + 1; 96 | } 97 | }); 98 | 99 | System.out.println("Skipped metrics (should be zero): " + stats[0]); 100 | System.out.println("Mismatched points count (should be zero): " + stats[2]); 101 | System.out.println("Processed metrics (should be non-zero): " + stats[1]); 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/tagstore/TagMatcher.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import org.apache.commons.lang3.StringUtils; 4 | 5 | /* 6 | * TagMatcher contains data about matching a tag with other tags in a query. It only contains config 7 | * data for a tag matcher but the actual implementation of tag matching happens in the tag store. 8 | * implementation of tag matching occurs inside the TagStore. 9 | */ 10 | public final class TagMatcher { 11 | 12 | public static final String WILDCARD = "*"; 13 | public final MatchType type; 14 | public final Tag tag; 15 | 16 | public TagMatcher(MatchType type, Tag tag) { 17 | if (type == null || tag == null) { 18 | throw new IllegalArgumentException("Tag or type can't be null"); 19 | } 20 | this.type = type; 21 | this.tag = tag; 22 | } 23 | 24 | @Override 25 | public boolean equals(Object o) { 26 | if (this == o) { 27 | return true; 28 | } 29 | if (o == null || getClass() != o.getClass()) { 30 | return false; 31 | } 32 | 33 | TagMatcher that = (TagMatcher) o; 34 | 35 | if (type != that.type) { 36 | return false; 37 | } 38 | return tag != null ? tag.equals(that.tag) : that.tag == null; 39 | 40 | } 41 | 42 | @Override 43 | public int hashCode() { 44 | int result = type != null ? type.hashCode() : 0; 45 | result = 31 * result + (tag != null ? tag.hashCode() : 0); 46 | return result; 47 | } 48 | 49 | /** 50 | * Exact matches all tag sets that contain a tag that exactly matches the given key and 51 | * value. 52 | * @param tag tag to match 53 | * @return a new TagMatcher object 54 | */ 55 | public static TagMatcher exactMatch(Tag tag) { 56 | return new TagMatcher(MatchType.EXACT, tag); 57 | } 58 | 59 | /** 60 | * Wildcard matches all tag sets that contain a tag that exactly matches the given key. 61 | * @param key the key to lookup 62 | * @param wildcardString 63 | * @return a new TagMatcher object 64 | */ 65 | public static TagMatcher wildcardMatch(String key, String wildcardString) { 66 | return createWildCardTagMatcher(key, wildcardString, MatchType.WILDCARD); 67 | } 68 | 69 | private static TagMatcher createWildCardTagMatcher(String key, String wildcardString, 70 | MatchType matchType) { 71 | 72 | if (StringUtils.isBlank(key) || StringUtils.isBlank(wildcardString)) { 73 | throw new IllegalArgumentException("Key or wildcard string cannot be null or empty"); 74 | } 75 | 76 | if (!wildcardString.contains(WILDCARD)) { 77 | throw new IllegalArgumentException("Wildcard string must contain an asterisk"); 78 | } 79 | 80 | return new TagMatcher(matchType, new Tag(key, wildcardString)); 81 | } 82 | 83 | public static TagMatcher iwildcardMatch(String key, String wildcardString) { 84 | return createWildCardTagMatcher(key, wildcardString, MatchType.IWILDCARD); 85 | } 86 | 87 | public static TagMatcher literalOrMatch(String key, String orLiteralString, 88 | boolean caseInsensitive) { 89 | 90 | return new TagMatcher(caseInsensitive ? MatchType.ILITERAL_OR : MatchType.LITERAL_OR, 91 | new Tag(key, orLiteralString)); 92 | } 93 | 94 | public static TagMatcher notLiteralOrMatch(String key, String orLiteralString, 95 | boolean caseInsensitive) { 96 | 97 | return new TagMatcher(caseInsensitive ? MatchType.NOT_ILITERAL_OR : MatchType.NOT_LITERAL_OR, 98 | new Tag(key, orLiteralString)); 99 | } 100 | 101 | public static TagMatcher regExMatch(String key, String regExString) { 102 | return new TagMatcher(MatchType.REGEXP, new Tag(key, regExString)); 103 | } 104 | 105 | @Override 106 | public String toString() { 107 | return "TagMatcher{" + "type=" + type + ", tag=" + tag + '}'; 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/metricstore/CachingVarBitTimeSeriesIterator.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore; 2 | 3 | import com.pinterest.yuvi.models.Point; 4 | import com.pinterest.yuvi.bitstream.BitStreamIterator; 5 | import com.pinterest.yuvi.bitstream.ParseException; 6 | 7 | import java.util.ArrayList; 8 | import java.util.List; 9 | 10 | /** 11 | * CachingVarBitTimeSeriesIterator is able to deserialize a delta-compressed time series into a 12 | * list of 13 | * points. The iterator can only read the bits at the time it is created. 14 | * 15 | * TODO: Pass in a reference to delta series to this iterator. 16 | * TODO: Implement Java iterator interface. 17 | * TODO: Separate the caching iterator and a non-caching iterator. 18 | * @see VarBitTimeSeries 19 | */ 20 | public class CachingVarBitTimeSeriesIterator implements TimeSeriesIterator { 21 | 22 | private final int count; 23 | private final BitStreamIterator timestamps; 24 | private final BitStreamIterator values; 25 | private long prevTimestamp; 26 | private long previousDelta; 27 | private int prevNumberOfLeadingZeros; 28 | private int prevNumberOfTrailingZeros; 29 | private long prevValue; 30 | private int idx; 31 | private List ts; 32 | 33 | /** 34 | * Create an iterator to read a given delta time series store. 35 | * @param count the number of items in the time series. 36 | * @param timestamps the compressed timestamps. 37 | * @param values the compressed values. 38 | */ 39 | public CachingVarBitTimeSeriesIterator(int count, BitStreamIterator timestamps, 40 | BitStreamIterator values) { 41 | this.count = count; 42 | this.timestamps = timestamps; 43 | this.values = values; 44 | } 45 | 46 | private void readFirst() { 47 | long t2 = timestamps.read(32); 48 | previousDelta = timestamps.read(14); 49 | prevTimestamp = t2 + previousDelta; 50 | 51 | prevValue = values.read(64); 52 | double v = Double.longBitsToDouble(prevValue); 53 | 54 | ts.add(new Point(prevTimestamp, v)); 55 | 56 | idx++; 57 | } 58 | 59 | private void readNext() { 60 | long deltaOfDelta; 61 | if (timestamps.tryRead(1, 0)) { 62 | deltaOfDelta = 0; 63 | } else if (timestamps.tryRead(2, 0b10)) { 64 | deltaOfDelta = timestamps.read(7) - 63; 65 | } else if (timestamps.tryRead(3, 0b110)) { 66 | deltaOfDelta = timestamps.read(9) - 255; 67 | } else if (timestamps.tryRead(4, 0b1110)) { 68 | deltaOfDelta = timestamps.read(12) - 2047; 69 | } else { 70 | timestamps.read(4); 71 | deltaOfDelta = timestamps.read(32) - 2147483647; 72 | } 73 | long delta = deltaOfDelta + previousDelta; 74 | long timeStamp = delta + prevTimestamp; 75 | prevTimestamp = timeStamp; 76 | previousDelta = delta; 77 | 78 | long xorValue; 79 | if (values.tryRead(1, 0)) { 80 | xorValue = 0; 81 | } else { 82 | long prefix = values.read(2); 83 | if (prefix == 0b11) { 84 | prevNumberOfLeadingZeros = (int) values.read(5); 85 | prevNumberOfTrailingZeros = (int) values.read(6); 86 | } 87 | int n = 64 - prevNumberOfLeadingZeros - prevNumberOfTrailingZeros; 88 | if (n < 0 || n > 64) { 89 | throw new ParseException("Invalid number of meaningful bits: " + n); 90 | } 91 | xorValue = values.read(n) << prevNumberOfTrailingZeros; 92 | } 93 | long v = prevValue ^ xorValue; // TODO: Encode -Inf into long turns it into NaN? 94 | double val = Double.longBitsToDouble(v); 95 | prevValue = v; 96 | 97 | ts.add(new Point(timeStamp, val)); 98 | 99 | idx++; 100 | } 101 | 102 | /** 103 | * Perform the decompression. May only be called once. 104 | * @throws Exception if the decompression fails. 105 | * @return the decompressed time series. 106 | */ 107 | public List getPoints() { 108 | ts = new ArrayList(count); 109 | if (count == 0) { 110 | return ts; 111 | } 112 | readFirst(); 113 | while (idx < count) { 114 | readNext(); 115 | } 116 | return ts; 117 | } 118 | } 119 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/tagstore/RoaringBitMapUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import org.roaringbitmap.RoaringBitmap; 4 | import org.roaringbitmap.buffer.MutableRoaringBitmap; 5 | 6 | import java.io.DataInputStream; 7 | import java.io.DataOutputStream; 8 | import java.io.IOException; 9 | import java.io.InputStream; 10 | import java.io.OutputStream; 11 | import java.nio.ByteBuffer; 12 | import java.nio.ByteOrder; 13 | 14 | /** 15 | * A utils class to convert between ByteBuffer and RoaringBitmap. 16 | * Code copied from https://github.com/RoaringBitmap/RoaringBitmap/blob/c304f92b3c31b37cc2961053ed37d2591165bcba/roaringbitmap/src/test/java/org/roaringbitmap/buffer/TestMemoryMapping.java 17 | */ 18 | public class RoaringBitMapUtils { 19 | 20 | public static ByteBuffer toByteBuffer(MutableRoaringBitmap rb) { 21 | // we add tests 22 | ByteBuffer outbb = ByteBuffer.allocate(rb.serializedSizeInBytes()); 23 | try { 24 | rb.serialize(new DataOutputStream(new ByteBufferBackedOutputStream(outbb))); 25 | } catch (IOException e) { 26 | e.printStackTrace(); 27 | } 28 | // 29 | outbb.flip(); 30 | outbb.order(ByteOrder.LITTLE_ENDIAN); 31 | return outbb; 32 | } 33 | 34 | public static ByteBuffer toByteBuffer(RoaringBitmap rb) { 35 | // we add tests 36 | ByteBuffer outbb = ByteBuffer.allocate(rb.serializedSizeInBytes()); 37 | try { 38 | rb.serialize(new DataOutputStream(new ByteBufferBackedOutputStream(outbb))); 39 | } catch (IOException e) { 40 | e.printStackTrace(); 41 | } 42 | // 43 | outbb.flip(); 44 | outbb.order(ByteOrder.LITTLE_ENDIAN); 45 | return outbb; 46 | } 47 | 48 | public static MutableRoaringBitmap toMutableRoaringBitmap(ByteBuffer bb) { 49 | MutableRoaringBitmap rb = new MutableRoaringBitmap(); 50 | try { 51 | rb.deserialize(new DataInputStream(new ByteBufferBackedInputStream(bb))); 52 | } catch (IOException e) { 53 | e.printStackTrace(); 54 | } 55 | return rb; 56 | } 57 | 58 | public static RoaringBitmap toRoaringBitmap(ByteBuffer bb) { 59 | RoaringBitmap rb = new RoaringBitmap(); 60 | try { 61 | rb.deserialize(new DataInputStream(new ByteBufferBackedInputStream(bb))); 62 | } catch (IOException e) { 63 | e.printStackTrace(); 64 | } 65 | return rb; 66 | } 67 | 68 | static class ByteBufferBackedOutputStream extends OutputStream { 69 | 70 | ByteBuffer buf; 71 | 72 | ByteBufferBackedOutputStream(ByteBuffer buf) { 73 | this.buf = buf; 74 | } 75 | 76 | @Override 77 | public synchronized void write(byte[] bytes) throws IOException { 78 | buf.put(bytes); 79 | } 80 | 81 | @Override 82 | public synchronized void write(byte[] bytes, int off, int len) throws IOException { 83 | buf.put(bytes, off, len); 84 | } 85 | 86 | @Override 87 | public synchronized void write(int b) throws IOException { 88 | buf.put((byte) b); 89 | } 90 | } 91 | 92 | static class ByteBufferBackedInputStream extends InputStream { 93 | 94 | ByteBuffer buf; 95 | 96 | ByteBufferBackedInputStream(ByteBuffer buf) { 97 | this.buf = buf; 98 | } 99 | 100 | @Override 101 | public int available() throws IOException { 102 | return buf.remaining(); 103 | } 104 | 105 | @Override 106 | public boolean markSupported() { 107 | return false; 108 | } 109 | 110 | @Override 111 | public int read() throws IOException { 112 | if (!buf.hasRemaining()) { 113 | return -1; 114 | } 115 | return 0xFF & buf.get(); 116 | } 117 | 118 | @Override 119 | public int read(byte[] bytes) throws IOException { 120 | int len = Math.min(bytes.length, buf.remaining()); 121 | buf.get(bytes, 0, len); 122 | return len; 123 | } 124 | 125 | @Override 126 | public int read(byte[] bytes, int off, int len) throws IOException { 127 | len = Math.min(len, buf.remaining()); 128 | buf.get(bytes, off, len); 129 | return len; 130 | } 131 | 132 | @Override 133 | public long skip(long n) { 134 | int len = Math.min((int) n, buf.remaining()); 135 | buf.position(buf.position() + (int) n); 136 | return len; 137 | } 138 | } 139 | } 140 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/chunk/ChunkManagerDataIntegrityChecker.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.models.TimeSeries; 4 | import com.pinterest.yuvi.tagstore.Query; 5 | 6 | import java.io.IOException; 7 | import java.nio.charset.Charset; 8 | import java.nio.file.Files; 9 | import java.nio.file.Path; 10 | import java.nio.file.Paths; 11 | import java.util.HashMap; 12 | import java.util.List; 13 | import java.util.stream.Stream; 14 | 15 | /** 16 | * This program takes a raw metrics file as input, ingests the data in it into a Yuvi ChunkManager, 17 | * reads it back and compares it to input data. 18 | * 19 | * For supporting large files, it compares the number of data points per metric instead of 20 | * actual data points. 21 | */ 22 | public class ChunkManagerDataIntegrityChecker { 23 | 24 | private final Path filePath; 25 | private final int tagStoreSize; 26 | private final boolean printErrorKey; 27 | 28 | public ChunkManagerDataIntegrityChecker() { 29 | this(System.getProperty("metricsData"), System.getProperty("tagStoreSize", "1000000")); 30 | } 31 | 32 | public ChunkManagerDataIntegrityChecker(String filePathName, String tagStoreSize) { 33 | if (filePathName == null || filePathName.isEmpty()) { 34 | throw new IllegalArgumentException("filePathName can't be empty"); 35 | } 36 | 37 | filePath = Paths.get(filePathName); 38 | this.tagStoreSize = new Integer(tagStoreSize); 39 | this.printErrorKey = new Boolean(System.getProperty("printErrorKey")); 40 | } 41 | 42 | public static void main(String[] args) throws IOException { 43 | ChunkManagerDataIntegrityChecker checker = new ChunkManagerDataIntegrityChecker(); 44 | final int[] stats = checker.checkData(); 45 | if (stats[0] == 0 && stats[2] == 0) { 46 | System.exit(0); 47 | } else { 48 | System.exit(-1); 49 | } 50 | } 51 | 52 | int[] checkData() throws IOException { 53 | ChunkManager store = new ChunkManager("test", tagStoreSize); 54 | 55 | HashMap counts = new HashMap(); 56 | 57 | try (Stream lines = Files.lines(filePath, Charset.defaultCharset())) { 58 | lines.forEachOrdered(line -> { 59 | if (line == null || line.isEmpty()) { 60 | return; 61 | } 62 | 63 | try { 64 | String[] words = line.split(" "); 65 | String metricName = words[1]; 66 | if (metricName != null && !metricName.isEmpty()) { 67 | store.addMetric(line); 68 | if (counts.containsKey(metricName)) { 69 | counts.put(metricName, counts.get(metricName) + 1); 70 | } else { 71 | counts.put(metricName, 1); 72 | } 73 | } else { 74 | System.out.println(metricName + " " + line); 75 | } 76 | } catch (Exception e) { 77 | System.out.println("error: " + e.getMessage()); 78 | e.printStackTrace(); 79 | } 80 | }); 81 | } 82 | 83 | checkMetrics(store, counts); 84 | 85 | store.toOffHeapChunkMap(); 86 | System.out.println("Converted to off heap store"); 87 | final int[] offHeapStats = checkMetrics(store, counts); 88 | 89 | return offHeapStats; 90 | } 91 | 92 | private int[] checkMetrics(ChunkManager store, HashMap counts) { 93 | final int[] stats = new int[3]; 94 | stats[0] = 0; 95 | stats[1] = 0; 96 | counts.keySet().stream().forEach(k -> { 97 | try { 98 | List 99 | timeSeriesList = store.query(Query.parse(k), 0, Long.MAX_VALUE, QueryAggregation.NONE); 100 | int pointsCount = timeSeriesList.stream().mapToInt(l -> l.getPoints().size()).sum(); 101 | if (counts.get(k) != pointsCount) { 102 | stats[2] = stats[2] + 1; 103 | if (printErrorKey) { 104 | System.out.println("Error key: " + k); 105 | } 106 | } 107 | stats[1] = stats[1] + 1; 108 | } catch (Exception e) { 109 | stats[0] = stats[0] + 1; 110 | } 111 | }); 112 | 113 | System.out.println("Skipped metrics (should be zero): " + stats[0]); 114 | System.out.println("Mismatched points count (should be zero): " + stats[2]); 115 | System.out.println("Processed metrics (should be non-zero): " + stats[1]); 116 | return stats; 117 | } 118 | } 119 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/tagstore/Metric.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.tagstore; 2 | 3 | import org.apache.commons.lang3.StringUtils; 4 | 5 | import java.util.ArrayList; 6 | import java.util.Collections; 7 | import java.util.List; 8 | import java.util.TreeSet; 9 | import java.util.stream.Collectors; 10 | 11 | /** 12 | * A ASCII metric metricName. The metric name contains a metric name and a list of tags. Each of the 13 | * tag is of the form key=value. 14 | */ 15 | public final class Metric { 16 | 17 | public static final char TAG_DELIMITER = '='; 18 | public static final char METRIC_SEPARATOR = ' '; 19 | 20 | public final String metricName; 21 | 22 | public final List tags; 23 | 24 | // This is metric name and sorted raw tag set. It uniquely identifies a metric. 25 | public final String fullMetricName; 26 | 27 | /** 28 | * A metric class to store the metric. 29 | * raw tags are key=value ascii string pairs. 30 | * 31 | * Pre-compute fullMetricName since it is needed by invertedTagStore and is also useful for 32 | * debugging. In future, this field 33 | * 34 | * If there is memory pressure, consider storing rawTags instead of Tag objects. 35 | * 36 | * @param metricName 37 | * @param rawTags 38 | * TODO: To optimize further, pass in rawTags that are sorted and trimmed. 39 | */ 40 | public Metric(final String metricName, final List rawTags) { 41 | if (StringUtils.isNotBlank(metricName)) { 42 | this.metricName = metricName; 43 | } else { 44 | throw new IllegalArgumentException("Invalid metric name"); 45 | } 46 | try { 47 | this.tags = parseTags(rawTags); 48 | } catch (Exception e) { 49 | throw new IllegalArgumentException("Error parsing tags" + rawTags, e); 50 | } 51 | 52 | this.fullMetricName = getFullMetricName(rawTags); 53 | } 54 | 55 | /** 56 | * Parse the tags and return a list of tags. The purpose of this method is to provide a single 57 | * method for parsing the tags. 58 | * Tag class can validate the tag key and value format. 59 | * @param rawTags a list of strings of the form "tag1=value1 tag2=value2..." 60 | */ 61 | public static List parseTags(List rawTags) { 62 | ArrayList tags = new ArrayList<>(rawTags.size()); 63 | for (String rawTag : rawTags) { 64 | if (rawTag != null && !rawTag.isEmpty()) { 65 | tags.add(Tag.parseTag(rawTag.trim())); 66 | } 67 | } 68 | return tags; 69 | } 70 | 71 | private String getFullMetricName(List rawTags) { 72 | TreeSet sortedTags = new TreeSet<>(); 73 | for (String rawTag : rawTags) { 74 | if (rawTag != null && !rawTag.isEmpty()) { 75 | sortedTags.add(rawTag.trim()); 76 | } 77 | } 78 | 79 | if (rawTags.isEmpty()) { 80 | return metricName; 81 | } else { 82 | StringBuilder fullMetricName = new StringBuilder(250); 83 | fullMetricName.append(metricName); 84 | for (String tag : sortedTags) { 85 | fullMetricName.append(METRIC_SEPARATOR); 86 | fullMetricName.append(tag); 87 | } 88 | 89 | return fullMetricName.toString(); 90 | } 91 | } 92 | 93 | @Override 94 | public boolean equals(Object o) { 95 | if (this == o) { 96 | return true; 97 | } 98 | if (o == null || getClass() != o.getClass()) { 99 | return false; 100 | } 101 | 102 | Metric metric = (Metric) o; 103 | 104 | if (metricName != null ? !metricName.equals(metric.metricName) : metric.metricName != null) { 105 | return false; 106 | } 107 | if (tags != null ? !tags.equals(metric.tags) : metric.tags != null) { 108 | return false; 109 | } 110 | return fullMetricName != null ? fullMetricName.equals(metric.fullMetricName) 111 | : metric.fullMetricName == null; 112 | 113 | } 114 | 115 | @Override 116 | public int hashCode() { 117 | int result = metricName != null ? metricName.hashCode() : 0; 118 | result = 31 * result + (tags != null ? tags.hashCode() : 0); 119 | result = 31 * result + (fullMetricName != null ? fullMetricName.hashCode() : 0); 120 | return result; 121 | } 122 | 123 | @Override 124 | public String toString() { 125 | return "Metric{" 126 | + "metricName='" + metricName + '\'' 127 | + ", tags=" + tags 128 | + ", fullMetricName='" + fullMetricName + '\'' 129 | + '}'; 130 | } 131 | } 132 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/metricstore/VarBitMetricStoreBenchmark.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore; 2 | 3 | import com.pinterest.yuvi.tagstore.Metric; 4 | 5 | import org.openjdk.jmh.annotations.Benchmark; 6 | import org.openjdk.jmh.annotations.BenchmarkMode; 7 | import org.openjdk.jmh.annotations.Fork; 8 | import org.openjdk.jmh.annotations.Level; 9 | import org.openjdk.jmh.annotations.Measurement; 10 | import org.openjdk.jmh.annotations.Mode; 11 | import org.openjdk.jmh.annotations.OutputTimeUnit; 12 | import org.openjdk.jmh.annotations.Param; 13 | import org.openjdk.jmh.annotations.Scope; 14 | import org.openjdk.jmh.annotations.Setup; 15 | import org.openjdk.jmh.annotations.State; 16 | import org.openjdk.jmh.annotations.Warmup; 17 | import org.openjdk.jmh.infra.Blackhole; 18 | import org.openjdk.jmh.runner.Runner; 19 | import org.openjdk.jmh.runner.RunnerException; 20 | import org.openjdk.jmh.runner.options.Options; 21 | import org.openjdk.jmh.runner.options.OptionsBuilder; 22 | 23 | import java.io.IOException; 24 | import java.nio.charset.Charset; 25 | import java.nio.file.Files; 26 | import java.nio.file.Path; 27 | import java.nio.file.Paths; 28 | import java.util.Arrays; 29 | import java.util.HashMap; 30 | import java.util.List; 31 | import java.util.concurrent.TimeUnit; 32 | import java.util.stream.Stream; 33 | 34 | @BenchmarkMode(Mode.AverageTime) 35 | @OutputTimeUnit(TimeUnit.MICROSECONDS) 36 | @Warmup(iterations = 5, time = 1, timeUnit = TimeUnit.MICROSECONDS) 37 | @Measurement(iterations = 10, time = 1, timeUnit = TimeUnit.MICROSECONDS) 38 | @Fork(1) 39 | @State(Scope.Benchmark) 40 | public class VarBitMetricStoreBenchmark { 41 | 42 | private Path filePath = Paths.get(System.getProperty("metricsData")); 43 | final int[] metricCounter = {0}; 44 | 45 | HashMap metricidHashMap = new HashMap(); 46 | VarBitMetricStore store = new VarBitMetricStore(10000000); 47 | 48 | public static void main(String[] args) throws RunnerException { 49 | Options opt = new OptionsBuilder() 50 | .include(VarBitMetricStoreBenchmark.class.getSimpleName()) 51 | .build(); 52 | new Runner(opt).run(); 53 | } 54 | 55 | @Param({"1", "5", "10", "50", "100", "500", "1000", "5000", "10000"}) 56 | public int fetchCount; 57 | 58 | 59 | @Setup 60 | public void setup() { 61 | try { 62 | load(); 63 | } catch (IOException e) { 64 | e.printStackTrace(); 65 | } 66 | } 67 | 68 | @Benchmark 69 | public void fetch(Blackhole bh) { 70 | bh.consume(fetchN(2 + fetchCount)); 71 | } 72 | 73 | private int fetchN(int n) { 74 | int size = 0; 75 | for (int i = 2; i < n; i++) { 76 | size = size + store.getSeries(2).size(); 77 | } 78 | return size; 79 | } 80 | 81 | public void load() throws IOException { 82 | // Get metric name and put it in a map and assign it a number. 83 | // Add ts and value to metric store with that number. 84 | try (Stream lines = Files.lines(filePath, Charset.defaultCharset())) { 85 | lines.forEachOrdered(line -> { 86 | if (line == null || line.isEmpty()) { // Ignore empty lines 87 | return; 88 | } 89 | try { 90 | String[] metricParts = line.split(" "); 91 | if (metricParts.length > 1 && metricParts[0].equals("put")) { 92 | String metricName = metricParts[1].trim(); 93 | List rawTags = Arrays.asList(metricParts).subList(4, metricParts.length); 94 | Metric metric = new Metric(metricName, rawTags); 95 | long ts = Long.parseLong(metricParts[2].trim()); 96 | double value = Double.parseDouble(metricParts[3].trim()); 97 | 98 | // System.out.println(metric.fullMetricName); 99 | 100 | int id = -1; 101 | if (metricidHashMap.containsKey(metric.fullMetricName)) { 102 | id = metricidHashMap.get(metric.fullMetricName); 103 | } else { 104 | metricCounter[0] = metricCounter[0] + 1; 105 | metricidHashMap.put(metric.fullMetricName, metricCounter[0]); 106 | id = metricCounter[0]; 107 | } 108 | 109 | store.addPoint(id, ts, value); 110 | } 111 | } catch (Exception e) { 112 | // System.out.println("Error ingesting line " + line + " with exception " + e.getMessage()); 113 | } 114 | }); 115 | } 116 | metricidHashMap.clear(); 117 | // System.out.println("Metric counter size: " + metricCounter[0]); 118 | } 119 | } 120 | -------------------------------------------------------------------------------- /benchmarks/src/main/java/com/pinterest/yuvi/chunk/ChunkManagerQueryBenchmark.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.pinterest.yuvi.models.TimeSeries; 4 | import com.pinterest.yuvi.writer.FileMetricWriter; 5 | import com.pinterest.yuvi.tagstore.Query; 6 | 7 | import org.openjdk.jmh.annotations.Benchmark; 8 | import org.openjdk.jmh.annotations.BenchmarkMode; 9 | import org.openjdk.jmh.annotations.Fork; 10 | import org.openjdk.jmh.annotations.Level; 11 | import org.openjdk.jmh.annotations.Measurement; 12 | import org.openjdk.jmh.annotations.Mode; 13 | import org.openjdk.jmh.annotations.OutputTimeUnit; 14 | import org.openjdk.jmh.annotations.Scope; 15 | import org.openjdk.jmh.annotations.Setup; 16 | import org.openjdk.jmh.annotations.State; 17 | import org.openjdk.jmh.annotations.Warmup; 18 | import org.openjdk.jmh.infra.Blackhole; 19 | import org.openjdk.jmh.runner.Runner; 20 | import org.openjdk.jmh.runner.RunnerException; 21 | import org.openjdk.jmh.runner.options.Options; 22 | import org.openjdk.jmh.runner.options.OptionsBuilder; 23 | 24 | import java.nio.file.Path; 25 | import java.nio.file.Paths; 26 | import java.util.List; 27 | import java.util.concurrent.TimeUnit; 28 | 29 | @BenchmarkMode(Mode.AverageTime) 30 | @OutputTimeUnit(TimeUnit.MILLISECONDS) 31 | @Warmup(iterations = 2, time = 1, timeUnit = TimeUnit.MILLISECONDS) 32 | @Measurement(iterations = 8, time = 1, timeUnit = TimeUnit.MILLISECONDS) 33 | @Fork(1) 34 | @State(Scope.Thread) 35 | public class ChunkManagerQueryBenchmark { 36 | 37 | private Path filePath = Paths.get(System.getProperty("metricsData")); 38 | private long startTs = 1489637603L; 39 | private long endTs = 1489809195L; 40 | private FileMetricWriter metricWriter; 41 | 42 | public static void main(String[] args) throws RunnerException { 43 | Options opt = new OptionsBuilder() 44 | .include(ChunkManagerQueryBenchmark.class.getSimpleName()) 45 | .build(); 46 | new Runner(opt).run(); 47 | } 48 | 49 | @Setup(Level.Trial) 50 | public void setup() { 51 | ChunkManager chunkManager = new ChunkManager("test", 1_000_000); 52 | metricWriter = new FileMetricWriter(filePath, chunkManager); 53 | metricWriter.start(); 54 | // Convert all data to offHeap 55 | chunkManager.toOffHeapChunkMap(); 56 | } 57 | 58 | @Benchmark 59 | public void queryByMetricName(Blackhole bh) throws Exception { 60 | List timeseries = 61 | metricWriter.getChunkManager().query( 62 | Query.parse("tc.proc.stat.cpu.total.coreapp-ngapi-prod"), 63 | startTs, 64 | endTs, QueryAggregation.NONE); 65 | System.out.println("timeseries size: " + timeseries.size()); 66 | System.out.println( 67 | "points size: " + timeseries.stream().mapToInt(series -> series.getPoints().size()).sum()); 68 | bh.consume(timeseries.size()); 69 | } 70 | 71 | @Benchmark 72 | public void queryByTag(Blackhole bh) throws Exception { 73 | List timeseries = 74 | metricWriter.getChunkManager().query( 75 | Query.parse("tc.proc.stat.cpu.total.coreapp-ngapi-prod ec2_zone=us-east-1d"), 76 | startTs, 77 | endTs, QueryAggregation.NONE); 78 | System.out.println("timeseries size: " + timeseries.size()); 79 | System.out.println( 80 | "points size: " + timeseries.stream().mapToInt(series -> series.getPoints().size()).sum()); 81 | bh.consume(timeseries.size()); 82 | } 83 | 84 | @Benchmark 85 | public void queryByWildTag(Blackhole bh) throws Exception { 86 | List timeseries = 87 | metricWriter.getChunkManager().query( 88 | Query.parse("tc.proc.stat.cpu.total.coreapp-ngapi-prod host=*"), 89 | startTs, 90 | endTs, QueryAggregation.NONE); 91 | System.out.println("timeseries size: " + timeseries.size()); 92 | System.out.println( 93 | "points size: " + timeseries.stream().mapToInt(series -> series.getPoints().size()).sum()); 94 | bh.consume(timeseries.size()); 95 | } 96 | 97 | @Benchmark 98 | public void queryByHost(Blackhole bh) throws Exception { 99 | List timeseries = 100 | metricWriter.getChunkManager().query( 101 | Query.parse( 102 | "tc.proc.stat.cpu.total.coreapp-ngapi-prod host=coreapp-ngapi-prod-0a018feb"), 103 | startTs, 104 | endTs, QueryAggregation.NONE); 105 | System.out.println("timeseries size: " + timeseries.size()); 106 | System.out.println( 107 | "points size: " + timeseries.stream().mapToInt(series -> series.getPoints().size()).sum()); 108 | bh.consume(timeseries.size()); 109 | } 110 | } 111 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/metricstore/VarBitMetricStore.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore; 2 | 3 | import com.pinterest.yuvi.chunk.ReadOnlyChunkInsertionException; 4 | import com.pinterest.yuvi.models.Point; 5 | 6 | import org.slf4j.Logger; 7 | import org.slf4j.LoggerFactory; 8 | 9 | import java.util.ArrayList; 10 | import java.util.Collections; 11 | import java.util.HashMap; 12 | import java.util.List; 13 | import java.util.Map; 14 | import java.util.concurrent.locks.ReentrantReadWriteLock; 15 | import java.util.function.Function; 16 | import java.util.stream.Collectors; 17 | 18 | 19 | /** 20 | * Implementation of MetricsStore that stores the time-series in compressed byte-arrays. 21 | */ 22 | public class VarBitMetricStore implements MetricStore { 23 | 24 | private static Logger LOG = LoggerFactory.getLogger(VarBitMetricStore.class); 25 | 26 | // TODO: Tune the default metrics size 27 | private static final int DEFAULT_METRIC_STORE_SIZE = 10_000; 28 | 29 | private HashMap series; 30 | private final ReentrantReadWriteLock mu; 31 | private boolean readOnly; 32 | public VarBitMetricStore() { 33 | this(DEFAULT_METRIC_STORE_SIZE); 34 | } 35 | 36 | /** 37 | * Create an empty metric store. 38 | */ 39 | public VarBitMetricStore(int initialSize) { 40 | series = new HashMap<>(initialSize); 41 | mu = new ReentrantReadWriteLock(); 42 | readOnly = false; 43 | 44 | LOG.info("Created a VarBitMetricStore with size {} and readOnly {}.", initialSize, readOnly); 45 | } 46 | 47 | @Override 48 | public List getSeries(long uuid) { 49 | mu.readLock().lock(); 50 | try { 51 | VarBitTimeSeries s = series.get(uuid); 52 | if (s == null) { 53 | return Collections.emptyList(); 54 | } 55 | return s.read().getPoints(); 56 | } finally { 57 | mu.readLock().unlock(); 58 | } 59 | } 60 | 61 | @Override 62 | public void addPoint(long uuid, long ts, double val) { 63 | if (!readOnly) { 64 | // Grab read lock for short path. 65 | VarBitTimeSeries s; 66 | mu.readLock().lock(); 67 | try { 68 | s = series.get(uuid); 69 | } finally { 70 | mu.readLock().unlock(); 71 | } 72 | if (s == null) { 73 | // Retry with write lock if short path failed. 74 | mu.writeLock().lock(); 75 | try { 76 | s = series.get(uuid); 77 | if (s == null) { 78 | s = new VarBitTimeSeries(); 79 | series.put(uuid, s); 80 | } 81 | } finally { 82 | mu.writeLock().unlock(); 83 | } 84 | } 85 | s.append(ts, val); 86 | } else { 87 | throw new ReadOnlyChunkInsertionException("Inserting a metric into a read only store for " 88 | + "uuid " + uuid + " ts " + ts + " val " + val); 89 | } 90 | } 91 | 92 | private ArrayList getUuids() throws Exception { 93 | // Copy the keys so that we don't hold the readLock for too long. 94 | mu.readLock().lock(); 95 | try { 96 | return new ArrayList(series.keySet()); 97 | } finally { 98 | mu.readLock().unlock(); 99 | } 100 | } 101 | 102 | @Override 103 | public Map getStats() { 104 | Map stats = new HashMap<>(); 105 | stats.put("MetricCount", new Double(series.size())); 106 | List> tsStats = 107 | series.values().stream().map(ts -> ts.getStats()).collect(Collectors.toList()); 108 | 109 | stats.put("TimeStampSizeDistribution", 110 | tsStats.stream().map(ts -> ts.get("timestamps_dataLength")) 111 | .collect(Collectors.groupingBy(Function.identity(), Collectors.counting()))); 112 | 113 | stats.put("ValueSizeDistribution", 114 | tsStats.stream().map(ts -> ts.get("values_dataLength")) 115 | .collect(Collectors.groupingBy(Function.identity(), Collectors.counting()))); 116 | 117 | stats.put("TimeStampByteSize", 118 | tsStats.stream().mapToLong(ts -> ts.get("timestamps_dataSize").longValue()).sum()); 119 | 120 | stats.put("ValuesByteSize", 121 | tsStats.stream().mapToLong(ts -> ts.get("values_dataSize").longValue()).sum()); 122 | 123 | return stats; 124 | } 125 | 126 | @Override 127 | public Map getSeriesMap() { 128 | return series; 129 | } 130 | 131 | @Override 132 | public void close() { 133 | this.series = null; 134 | } 135 | 136 | public void setReadOnly(boolean readOnly) { 137 | this.readOnly = readOnly; 138 | LOG.info("Chunk readOnly status is {}", this.readOnly); 139 | } 140 | 141 | @Override 142 | public boolean isReadOnly() { 143 | return readOnly; 144 | } 145 | } 146 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/metricstore/offheap/OffHeapVarBitMetricStoreTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore.offheap; 2 | 3 | import static org.junit.Assert.assertEquals; 4 | import static org.junit.Assert.assertTrue; 5 | 6 | import com.pinterest.yuvi.metricstore.MetricStore; 7 | import com.pinterest.yuvi.metricstore.VarBitMetricStore; 8 | import com.pinterest.yuvi.metricstore.VarBitTimeSeries; 9 | import com.pinterest.yuvi.models.Point; 10 | 11 | import net.openhft.chronicle.hash.ChronicleHashClosedException; 12 | import org.junit.Test; 13 | 14 | import java.time.Instant; 15 | import java.util.List; 16 | import java.util.Map; 17 | 18 | public class OffHeapVarBitMetricStoreTest { 19 | 20 | final static double delta = 0.00001; 21 | final String testFileName = ""; 22 | 23 | @Test 24 | public void testEmpty() { 25 | MetricStore store = new OffHeapVarBitMetricStore(1, testFileName); 26 | assertTrue(store.getSeries(1).isEmpty()); 27 | assertTrue(store.getSeries(2).isEmpty()); 28 | } 29 | 30 | @Test(expected = UnsupportedOperationException.class) 31 | public void testReadOnlyStore() { 32 | MetricStore store = new OffHeapVarBitMetricStore(1, testFileName); 33 | store.addPoint(1, 1, 1); 34 | } 35 | 36 | @Test(expected = ChronicleHashClosedException.class) 37 | public void testCloseWithoutPersistedFile() { 38 | MetricStore heapStore = new VarBitMetricStore(); 39 | long uuid1 = 1; 40 | long ts = Instant.now().getEpochSecond(); 41 | double value = 100; 42 | 43 | heapStore.addPoint(uuid1, ts, value); 44 | OffHeapVarBitMetricStore offheapStore1 = 45 | OffHeapVarBitMetricStore.toOffHeapStore(getSeriesMap(heapStore), testFileName, ""); 46 | 47 | assertEquals(1, offheapStore1.getSeriesMap().size()); 48 | List points = offheapStore1.getSeries(uuid1); 49 | assertEquals(1, points.size()); 50 | assertEquals(ts, points.get(0).getTs()); 51 | assertEquals(value, points.get(0).getVal(), delta); 52 | 53 | offheapStore1.close(); 54 | offheapStore1.getSeriesMap().size(); 55 | } 56 | 57 | @Test 58 | public void testSimpleInserts() { 59 | MetricStore heapStore = new VarBitMetricStore(); 60 | long uuid1 = 1; 61 | long uuid2 = 2; 62 | long ts = Instant.now().getEpochSecond(); 63 | double value = 100; 64 | 65 | // 1 metric with 1 value. 66 | heapStore.addPoint(uuid1, ts, value); 67 | OffHeapVarBitMetricStore offheapStore1 = 68 | OffHeapVarBitMetricStore.toOffHeapStore(getSeriesMap(heapStore), testFileName, ""); 69 | 70 | assertEquals(1, offheapStore1.getSeriesMap().size()); 71 | List points = offheapStore1.getSeries(uuid1); 72 | assertEquals(1, points.size()); 73 | assertEquals(ts, points.get(0).getTs()); 74 | assertEquals(value, points.get(0).getVal(), delta); 75 | 76 | // 1 metric with 2 values. 77 | heapStore.addPoint(uuid1, ts + 1, value + 1); 78 | OffHeapVarBitMetricStore offheapStore2 = 79 | OffHeapVarBitMetricStore.toOffHeapStore(getSeriesMap(heapStore), testFileName, ""); 80 | List points2 = offheapStore2.getSeries(uuid1); 81 | assertEquals(2, points2.size()); 82 | assertEquals(ts, points2.get(0).getTs()); 83 | assertEquals(value, points2.get(0).getVal(), delta); 84 | assertEquals(ts + 1, points2.get(1).getTs()); 85 | assertEquals(value + 1, points2.get(1).getVal(), delta); 86 | 87 | // 2 metrics with 2 values each. 88 | heapStore.addPoint(uuid2, ts + 2, value + 2); 89 | OffHeapVarBitMetricStore offheapStore3 = 90 | OffHeapVarBitMetricStore.toOffHeapStore(getSeriesMap(heapStore), testFileName, ""); 91 | List points31 = offheapStore3.getSeries(uuid1); 92 | assertEquals(2, points31.size()); 93 | assertEquals(ts, points31.get(0).getTs()); 94 | assertEquals(value, points31.get(0).getVal(), delta); 95 | assertEquals(ts + 1, points31.get(1).getTs()); 96 | assertEquals(value + 1, points31.get(1).getVal(), delta); 97 | List points32 = offheapStore3.getSeries(uuid2); 98 | assertEquals(1, points32.size()); 99 | assertEquals(ts + 2, points32.get(0).getTs()); 100 | assertEquals(value + 2, points32.get(0).getVal(), delta); 101 | 102 | heapStore.addPoint(uuid2, ts + 3, value + 3); 103 | OffHeapVarBitMetricStore offheapStore4 = 104 | OffHeapVarBitMetricStore.toOffHeapStore(getSeriesMap(heapStore), testFileName, ""); 105 | List points41 = offheapStore4.getSeries(uuid1); 106 | assertEquals(2, points41.size()); 107 | assertEquals(ts, points41.get(0).getTs()); 108 | assertEquals(value, points41.get(0).getVal(), delta); 109 | assertEquals(ts + 1, points41.get(1).getTs()); 110 | assertEquals(value + 1, points41.get(1).getVal(), delta); 111 | List points42 = offheapStore4.getSeries(uuid2); 112 | assertEquals(2, points42.size()); 113 | assertEquals(ts + 2, points42.get(0).getTs()); 114 | assertEquals(value + 2, points42.get(0).getVal(), delta); 115 | assertEquals(ts + 3, points42.get(1).getTs()); 116 | assertEquals(value + 3, points42.get(1).getVal(), delta); 117 | } 118 | 119 | // TODO: Create tests for creating, using and deleting persisted file. 120 | 121 | @SuppressWarnings("unchecked") 122 | private Map getSeriesMap(MetricStore heapStore) { 123 | return (Map) heapStore.getSeriesMap(); 124 | } 125 | } 126 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/chunk/OffHeapChunkManagerTask.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import com.google.common.annotations.VisibleForTesting; 4 | import org.slf4j.Logger; 5 | import org.slf4j.LoggerFactory; 6 | 7 | import java.time.Instant; 8 | import java.util.ArrayList; 9 | import java.util.List; 10 | import java.util.Map; 11 | 12 | /** 13 | * Yuvi stores all the metrics data in memory, but there is a GC overhead to storing all the metrics 14 | * data in memory, so we move the metrics data that will be no longer updated to off heap store and 15 | * mark it read only to save GC overhead. Further, after stale data delay, the data is deleted from 16 | * the chunk manager. 17 | * 18 | * This class identifies chunks in chunk manager that can be moved off heap. A chunk can be moved 19 | * off heap if the current time is greater than the chunk end time. However, sometimes metrics may 20 | * be delayed because of transient issues. To ingest late arriving metrics, we wait a metricsDelay 21 | * amount of time before we start marking the chunk read-only and moving it to off heap. 22 | */ 23 | public class OffHeapChunkManagerTask implements Runnable { 24 | 25 | private static final Logger LOG = LoggerFactory.getLogger(OffHeapChunkManagerTask.class); 26 | 27 | public static final int DEFAULT_METRICS_DELAY_SECS = 15 * 60; // 15 minutes. 28 | public static final int DEFAULT_STALE_DATA_DELAY_SECS = 6 * 60 * 60; // 6 hours. 29 | 30 | // The time after the end time after which a chunk will be marked as read only 31 | private final int metricsDelay; 32 | private final int staleDataDelaySecs; 33 | private ChunkManager chunkManager; 34 | 35 | public OffHeapChunkManagerTask(ChunkManager chunkManager) { 36 | this(chunkManager, DEFAULT_METRICS_DELAY_SECS, DEFAULT_STALE_DATA_DELAY_SECS); 37 | } 38 | 39 | public OffHeapChunkManagerTask(ChunkManager chunkManager, 40 | int metricsDelaySecs, 41 | int staleDataDelaySecs) { 42 | 43 | this.chunkManager = chunkManager; 44 | this.metricsDelay = metricsDelaySecs; 45 | this.staleDataDelaySecs = staleDataDelaySecs; 46 | } 47 | 48 | @Override 49 | public void run() { 50 | try{ 51 | runAt(Instant.now()); 52 | } catch (Exception e) { 53 | LOG.error("Off heap chunk manager failed with an error", e); 54 | } 55 | } 56 | 57 | /** 58 | * Run the chunk manager tasks at an instant. Sometimes some metrics may be very late and will be 59 | * written to a chunk on the heap. It is wasteful to move this data off heap and then delete it. 60 | * So, first run deleteStaleData so we can delete stale data before we spend the work to move it 61 | * off heap. Further, deleting stale data first will also free resources faster. 62 | */ 63 | @VisibleForTesting 64 | void runAt(Instant instant) { 65 | LOG.info("Starting offHeapChunkManagerTask."); 66 | deleteStaleData(instant); 67 | detectReadOnlyChunks(instant); 68 | LOG.info("Finished offHeapChunkManagerTask."); 69 | } 70 | 71 | @VisibleForTesting 72 | int detectReadOnlyChunks(Instant startInstant) { 73 | int secondsToSubtract = this.metricsDelay; 74 | // cutOffTime = chunk end time + metrics delay. 75 | final long offHeapCutoffSecs = startInstant.minusSeconds(secondsToSubtract).getEpochSecond(); 76 | return detectChunksPastCutOff(offHeapCutoffSecs); 77 | } 78 | 79 | @VisibleForTesting 80 | int detectChunksPastCutOff(long offHeapCutoffSecs) { 81 | if (offHeapCutoffSecs <= 0) { 82 | throw new IllegalArgumentException("offHeapCutoffSecs can't be negative."); 83 | } 84 | 85 | LOG.info("offHeapCutOffSecs is {}", offHeapCutoffSecs); 86 | List> readOnlyChunks = new ArrayList<>(); 87 | for (Map.Entry chunkEntry: chunkManager.getChunkMap().entrySet()) { 88 | Chunk chunk = chunkEntry.getValue(); 89 | if (!chunk.isReadOnly() && offHeapCutoffSecs >= chunk.info().endTimeSecs) { 90 | readOnlyChunks.add(chunkEntry); 91 | } 92 | } 93 | 94 | LOG.info("Number of chunks past cut off: {}.", readOnlyChunks.size()); 95 | chunkManager.toReadOnlyChunks(readOnlyChunks); 96 | return readOnlyChunks.size(); 97 | } 98 | 99 | /** 100 | * Delete stale data when running at startInstant. 101 | */ 102 | int deleteStaleData(Instant startInstant) { 103 | final long staleCutoffSecs = startInstant.minusSeconds(this.staleDataDelaySecs).getEpochSecond(); 104 | return deleteStaleChunks(staleCutoffSecs); 105 | } 106 | 107 | /** 108 | * Delete all chunks that are older than the cut off seconds. 109 | */ 110 | int deleteStaleChunks(long staleDataCutoffSecs) { 111 | if (staleDataCutoffSecs <= 0) { 112 | throw new IllegalArgumentException("staleDateCutOffSecs can't be negative."); 113 | } 114 | 115 | LOG.info("stale data cut off secs is {}.", staleDataCutoffSecs); 116 | List> staleChunks = new ArrayList<>(); 117 | for (Map.Entry chunkEntry: chunkManager.getChunkMap().entrySet()) { 118 | Chunk chunk = chunkEntry.getValue(); 119 | if (chunk.info().endTimeSecs <= staleDataCutoffSecs) { 120 | staleChunks.add(chunkEntry); 121 | } 122 | } 123 | 124 | LOG.info("Number of stale chunks is: {}.", staleChunks.size()); 125 | chunkManager.removeStaleChunks(staleChunks); 126 | return staleChunks.size(); 127 | } 128 | } 129 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/bitstream/BitStream.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.bitstream; 2 | 3 | import com.google.common.annotations.VisibleForTesting; 4 | import com.google.common.primitives.Ints; 5 | 6 | import java.nio.ByteBuffer; 7 | import java.util.Collections; 8 | import java.util.HashMap; 9 | import java.util.Map; 10 | 11 | /** 12 | * A class that stores a sequence of bits in a binary array. Bits are written sequentially. The 13 | * underlying array grows automatically. This class is not thread safe. 14 | * TODO: Bounds checks on capacity, index and shift 15 | * TODO: Tweak initial capacity to reduce resize ops. 16 | */ 17 | public class BitStream { 18 | 19 | static final int DEFAULT_INITIAL_CAPACITY = 16; 20 | 21 | private static final double DEFAULT_CAPACITY_MULTIPLE = 2; 22 | 23 | private long[] data; 24 | private int capacity; 25 | private int index; 26 | private byte shift; 27 | 28 | BitStream() { 29 | this(DEFAULT_INITIAL_CAPACITY); 30 | } 31 | 32 | public BitStream(int initialCapacity) { 33 | this(new long[initialCapacity], initialCapacity, 0, (byte) 0); 34 | } 35 | 36 | public BitStream(long[] data, int len, int index, byte shift) { 37 | this.data = data; 38 | this.capacity = len; 39 | this.index = index; 40 | this.shift = shift; 41 | } 42 | 43 | /** 44 | * Return the number of bits that have been written. 45 | * @return number of bits written. 46 | */ 47 | int bitOffset() { 48 | return index * 64 + shift; 49 | } 50 | 51 | private long bitCapacity() { 52 | return capacity * 64; 53 | } 54 | 55 | private void resize() { 56 | int newSize = 57 | new Double(Math.floor((double) capacity * DEFAULT_CAPACITY_MULTIPLE)).intValue(); 58 | 59 | long[] newdata = new long[newSize]; 60 | for (int i = 0; i < capacity; i++) { 61 | newdata[i] = data[i]; 62 | } 63 | 64 | capacity = newSize; 65 | data = newdata; 66 | } 67 | 68 | private void reserve(int n) { 69 | while (bitCapacity() - bitOffset() < n) { 70 | resize(); 71 | } 72 | } 73 | 74 | /** 75 | * Append up to 64 bits to the array. 76 | * @param n the number of bits to append. Between 0 and 64. 77 | * @param v an integer containing the bits. The n least-significant bits are used. 78 | * TODO: test for value overflow also. 79 | */ 80 | public void write(int n, long v) { 81 | if (n < 1 || n > 64) { 82 | throw new IllegalArgumentException( 83 | String.format("Unable to write %s bits to value %d", n, v)); 84 | } 85 | 86 | reserve(n); 87 | long v1 = v << 64 - n >>> shift; 88 | data[index] = data[index] | v1; 89 | shift += n; 90 | if (shift >= 64) { 91 | shift -= 64; 92 | index++; 93 | if (shift != 0) { 94 | long v2 = v << 64 - shift; 95 | data[index] = data[index] | v2; 96 | } 97 | } 98 | } 99 | 100 | public Map getStats() { 101 | HashMap stats = new HashMap<>(); 102 | stats.put("dataLength", new Double(index)); 103 | stats.put("dataSize", new Double(getSerializedByteSize())); 104 | stats.put("capacity", new Double(capacity)); 105 | return Collections.unmodifiableMap(stats); 106 | } 107 | 108 | /** 109 | * Return an object to read the bit stream. Bits are immutable after they are written, so the 110 | * returned reader may be used on a separate thread. 111 | * @return a reader pointing to the data that has been written. 112 | */ 113 | public BitStreamIterator read() { 114 | return new BitStreamIterator(data, bitOffset()); 115 | } 116 | 117 | /** 118 | * Construct a new BitReader using the data in the given ByteBuffer. 119 | * @param buffer a buffer containing the data 120 | * @throws Exception if the parsing failed. 121 | * @return a new BitReader 122 | */ 123 | public static BitStream deserialize(ByteBuffer buffer) throws Exception { 124 | int validDataSize = buffer.getInt(); 125 | byte shift = buffer.get(); 126 | long[] data = new long[validDataSize]; 127 | for (int i = 0; i < validDataSize; i++) { 128 | data[i] = buffer.getLong(); 129 | } 130 | int index = shift == 0 ? validDataSize : validDataSize - 1; 131 | return new BitStream(data, validDataSize, index, shift); 132 | } 133 | 134 | /** 135 | * Write the data to a pre-allocated ByteBuffer. 136 | * @param buffer must have capacity greater or equal to serializedSize 137 | * @throws Exception if buffer is invalid 138 | */ 139 | public void serialize(ByteBuffer buffer) throws Exception { 140 | int validDataSize = getLastDataIndex(); 141 | buffer.putInt(validDataSize); 142 | buffer.put(shift); 143 | for (int i = 0; i < validDataSize; i++) { 144 | buffer.putLong(data[i]); 145 | } 146 | } 147 | 148 | public int getSerializedByteSize() { 149 | return Ints.BYTES // Size of index. 150 | + Byte.BYTES + // Size of shift 151 | Long.BYTES * getLastDataIndex(); // Size of long valid data 152 | } 153 | 154 | private int getLastDataIndex() { 155 | return shift == 0 ? index : index + 1; 156 | } 157 | 158 | @VisibleForTesting 159 | long[] getData() { 160 | return data; 161 | } 162 | 163 | @VisibleForTesting 164 | int getCapacity() { 165 | return capacity; 166 | } 167 | 168 | @VisibleForTesting 169 | int getIndex() { 170 | return index; 171 | } 172 | 173 | @VisibleForTesting 174 | byte getShift() { 175 | return shift; 176 | } 177 | } 178 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/metricstore/offheap/OffHeapVarBitMetricStore.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore.offheap; 2 | 3 | import com.pinterest.yuvi.metricstore.MetricStore; 4 | import com.pinterest.yuvi.metricstore.TimeSeriesIterator; 5 | import com.pinterest.yuvi.metricstore.VarBitTimeSeries; 6 | import com.pinterest.yuvi.models.Point; 7 | 8 | import net.openhft.chronicle.core.values.LongValue; 9 | import net.openhft.chronicle.map.ChronicleMap; 10 | import net.openhft.chronicle.map.ChronicleMapBuilder; 11 | import net.openhft.chronicle.values.Values; 12 | import org.slf4j.Logger; 13 | import org.slf4j.LoggerFactory; 14 | 15 | import java.io.File; 16 | import java.io.IOException; 17 | import java.nio.ByteBuffer; 18 | import java.util.Collections; 19 | import java.util.HashMap; 20 | import java.util.List; 21 | import java.util.Map; 22 | 23 | /** 24 | * The off heap metric store stores a snapshot of the metrics off heap. 25 | */ 26 | public class OffHeapVarBitMetricStore implements MetricStore { 27 | 28 | private static Logger LOG = LoggerFactory.getLogger(OffHeapVarBitMetricStore.class); 29 | 30 | private static final int DEFAULT_VALUE_SIZE = 2000; 31 | 32 | private Map timeSeries; 33 | 34 | private static final String offHeapNamePrefix = "yuvi_timeseries"; 35 | 36 | private final String chunkInfo; 37 | 38 | public OffHeapVarBitMetricStore(long size, String chunkInfo) { 39 | this(size, DEFAULT_VALUE_SIZE, chunkInfo); 40 | } 41 | 42 | public OffHeapVarBitMetricStore(long size, int valueSize, String chunkInfo) { 43 | this(size, valueSize, chunkInfo, ""); 44 | } 45 | 46 | public OffHeapVarBitMetricStore(long size, int valueSize, String chunkInfo, String dir) { 47 | this.chunkInfo = chunkInfo; 48 | 49 | ChronicleMapBuilder mapBuilder = ChronicleMap 50 | .of(LongValue.class, ByteBuffer.class) 51 | .entries(size) 52 | .averageValueSize(valueSize); 53 | 54 | if (chunkInfo != null && !chunkInfo.isEmpty() && !dir.isEmpty()) { 55 | File offHeapFile = new File(dir + "/" + offHeapNamePrefix + "_" + chunkInfo); 56 | try { 57 | timeSeries = mapBuilder.name(offHeapNamePrefix + "_" + chunkInfo) 58 | .createPersistedTo(offHeapFile); 59 | } catch (IOException e) { 60 | LOG.error("Failed to create an offheap store {} with error {}", offHeapFile, e.getMessage()); 61 | throw new IllegalArgumentException("Failed to create an off heap store.", e); 62 | } 63 | } else { 64 | timeSeries = mapBuilder.name(offHeapNamePrefix).create(); 65 | } 66 | LOG.info("Created an off heap metric store of size={} valueSize={} chunkInfo={} in dir={}", 67 | size, valueSize, chunkInfo, dir); 68 | } 69 | 70 | @Override 71 | public List getSeries(long uuid) { 72 | final LongValue key = Values.newHeapInstance(LongValue.class); 73 | key.setValue(uuid); 74 | if (timeSeries.containsKey(key)) { 75 | ByteBuffer serializedValues = timeSeries.get(key); 76 | TimeSeriesIterator iterator = VarBitTimeSeries.deserialize(serializedValues); 77 | return iterator.getPoints(); 78 | } else { 79 | return Collections.emptyList(); 80 | } 81 | } 82 | 83 | /** 84 | * Create an OffHeapMetricStore from a MetricStore. 85 | * TODO: We use the max value size for all values. But it can be tuned. 86 | */ 87 | public static OffHeapVarBitMetricStore toOffHeapStore(Map timeSeriesMap, 88 | String chunkInfo, String dataDirectory) { 89 | 90 | int maxSize = timeSeriesMap.values().stream() 91 | .mapToInt(series -> series.getSerializedByteSize()) 92 | .max() 93 | .getAsInt(); 94 | 95 | OffHeapVarBitMetricStore offHeapStore = 96 | new OffHeapVarBitMetricStore(timeSeriesMap.size(), maxSize, chunkInfo, dataDirectory); 97 | 98 | timeSeriesMap.entrySet().forEach(e -> { 99 | try { 100 | VarBitTimeSeries series = e.getValue(); 101 | 102 | int serializedByteSize = series.getSerializedByteSize(); 103 | ByteBuffer serializedTimeSeriesBuffer = ByteBuffer.allocate(serializedByteSize); 104 | series.serialize(serializedTimeSeriesBuffer); 105 | // This is needed because JVM is big-endian but linux native memory is little-endian. 106 | serializedTimeSeriesBuffer.flip(); 107 | offHeapStore.addPoint(e.getKey(), serializedTimeSeriesBuffer); 108 | } catch (Exception ex) { 109 | LOG.info("Moving entry {} in chunk {} to off heap failed with exception {}", 110 | e.getKey(), chunkInfo, ex); 111 | } 112 | }); 113 | return offHeapStore; 114 | } 115 | 116 | public void addPoint(long uuid, ByteBuffer series) { 117 | LongValue key = Values.newHeapInstance(LongValue.class); 118 | key.setValue(uuid); 119 | timeSeries.put(key, series); 120 | } 121 | 122 | @Override 123 | public void addPoint(long uuid, long ts, double val) { 124 | throw new UnsupportedOperationException("This is a read only metric store"); 125 | } 126 | 127 | @Override 128 | public Map getStats() { 129 | Map stats = new HashMap<>(); 130 | stats.put("MetricCount", new Double(timeSeries.size())); 131 | stats.put("TimeSeriesByteSize", 132 | timeSeries.values().stream().mapToInt(ts -> ts.capacity()).sum()); 133 | return stats; 134 | } 135 | 136 | @Override 137 | public Map getSeriesMap() { 138 | return timeSeries; 139 | } 140 | 141 | @Override 142 | public void close() { 143 | ChronicleMap timeSeries = (ChronicleMap) this.timeSeries; 144 | LOG.info("Closing chronicle map {}", chunkInfo); 145 | // Closing the timeSeries frees up all the resources associated with this map if there are no 146 | // other references to it. So, make sure all the references to the map are closed. 147 | timeSeries.close(); 148 | LOG.info("Closed chronicle map {}.", chunkInfo); 149 | } 150 | 151 | @Override 152 | public boolean isReadOnly() { 153 | return true; 154 | } 155 | 156 | @Override 157 | public void setReadOnly(boolean readOnly) { 158 | throw new UnsupportedOperationException("The store is read-only " + chunkInfo); 159 | } 160 | } 161 | -------------------------------------------------------------------------------- /benchmarks/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | com.pinterest.yuvi 8 | benchmarks 9 | 0.1-SNAPSHOT 10 | jar 11 | 12 | 13 | 14 | Apache License, Version 2.0 15 | https://www.apache.org/licenses/LICENSE-2.0.txt 16 | repo 17 | A business-friendly OSS license 18 | 19 | 20 | 21 | 22 | UTF-8 23 | 1.19 24 | 1.8 25 | benchmarks 26 | 27 | 28 | 29 | 30 | org.openjdk.jmh 31 | jmh-core 32 | ${jmh.version} 33 | 34 | 35 | org.openjdk.jmh 36 | jmh-generator-annprocess 37 | ${jmh.version} 38 | provided 39 | 40 | 41 | com.pinterest.yuvi 42 | yuvi 43 | 0.1-SNAPSHOT 44 | 45 | 46 | org.slf4j 47 | slf4j-log4j12 48 | 1.7.7 49 | 50 | 51 | 52 | 53 | 54 | 55 | org.apache.maven.plugins 56 | maven-compiler-plugin 57 | 3.1 58 | 59 | ${javac.target} 60 | ${javac.target} 61 | ${javac.target} 62 | 63 | 64 | 65 | org.apache.maven.plugins 66 | maven-shade-plugin 67 | 2.2 68 | 69 | 70 | package 71 | 72 | shade 73 | 74 | 75 | ${uberjar.name} 76 | 77 | 79 | org.openjdk.jmh.Main 80 | 81 | 82 | 83 | 84 | 88 | *:* 89 | 90 | META-INF/*.SF 91 | META-INF/*.DSA 92 | META-INF/*.RSA 93 | 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | maven-clean-plugin 105 | 2.5 106 | 107 | 108 | maven-deploy-plugin 109 | 2.8.1 110 | 111 | 112 | maven-install-plugin 113 | 2.5.1 114 | 115 | 116 | maven-jar-plugin 117 | 2.4 118 | 119 | 120 | maven-javadoc-plugin 121 | 2.9.1 122 | 123 | 124 | maven-resources-plugin 125 | 2.6 126 | 127 | 128 | maven-site-plugin 129 | 3.3 130 | 131 | 132 | maven-source-plugin 133 | 2.2.1 134 | 135 | 136 | maven-surefire-plugin 137 | 2.17 138 | 139 | 140 | 141 | 142 | 143 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/metricandtagstore/MetricsAndTagStoreImplTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricandtagstore; 2 | 3 | import static org.junit.Assert.assertThat; 4 | 5 | import com.pinterest.yuvi.metricstore.VarBitMetricStore; 6 | import com.pinterest.yuvi.models.Point; 7 | import com.pinterest.yuvi.models.TimeSeries; 8 | import com.pinterest.yuvi.tagstore.InvertedIndexTagStore; 9 | import com.pinterest.yuvi.tagstore.Metric; 10 | import com.pinterest.yuvi.tagstore.Query; 11 | 12 | import junit.framework.TestCase; 13 | import org.hamcrest.collection.IsIterableContainingInOrder; 14 | 15 | import java.util.Arrays; 16 | import java.util.Collections; 17 | import java.util.List; 18 | 19 | public class MetricsAndTagStoreImplTest extends TestCase { 20 | 21 | private MetricsAndTagStoreImpl ms; 22 | 23 | private final long ts = 100L; 24 | private final double value = 10; 25 | 26 | @Override 27 | public void setUp() { 28 | ms = new MetricsAndTagStoreImpl(new InvertedIndexTagStore(), new VarBitMetricStore()); 29 | } 30 | 31 | public void testBasicsInYuviStore() { 32 | // query empty store. 33 | final String testMetricName1 = "testMetric1"; 34 | final List testTags1 = Arrays.asList("host=h1", "dc=dc1"); 35 | assertTrue(ms.getSeries(new Query("test", Collections.emptyList())).isEmpty()); 36 | 37 | // 1 metric 1 data point 38 | final Metric testMetric1 = new Metric(testMetricName1, testTags1); 39 | final String expectedMetricName1 = testMetricName1 + " dc=dc1 host=h1"; 40 | final String queryTagString = " host=h1 dc=dc1"; 41 | 42 | ms.addPoint(testMetric1, ts, value); 43 | 44 | assertTrue(ms.getSeries(Query.parse("test")).isEmpty()); 45 | assertTrue(ms.getSeries(Query.parse("test host=h1")).isEmpty()); 46 | assertTrue(ms.getSeries(Query.parse("test host=h1 dc=dc1")).isEmpty()); 47 | 48 | final List 49 | series1 = 50 | ms.getSeries(Query.parse(testMetricName1 + queryTagString)); 51 | assertEquals(1, series1.size()); 52 | assertEquals(expectedMetricName1, series1.get(0).getMetric()); 53 | assertEquals(1, series1.get(0).getPoints().size()); 54 | assertThat(series1.get(0).getPoints(), 55 | IsIterableContainingInOrder.contains(new Point(ts, value))); 56 | 57 | // 1 metric 2 points 58 | ms.addPoint(testMetric1, ts * 2, value * 2); 59 | 60 | assertTrue(ms.getSeries(Query.parse("test")).isEmpty()); 61 | assertTrue(ms.getSeries(Query.parse("test host=h1")).isEmpty()); 62 | assertTrue(ms.getSeries(Query.parse("test host=h1 dc=dc1")).isEmpty()); 63 | 64 | final List 65 | series2 = 66 | ms.getSeries(Query.parse(testMetricName1 + queryTagString)); 67 | assertEquals(1, series2.size()); 68 | assertEquals(expectedMetricName1, series2.get(0).getMetric()); 69 | assertEquals(2, series2.get(0).getPoints().size()); 70 | List expectedPoints2 = 71 | Arrays.asList(new Point(ts, value), new Point(ts * 2, value * 2)); 72 | final TimeSeries timeseries12 = new TimeSeries(expectedMetricName1, expectedPoints2); 73 | assertThat(series2, IsIterableContainingInOrder.contains(timeseries12)); 74 | 75 | // 2 metrics 2 points 76 | final String testMetricName2 = "testMetric2"; 77 | final Metric testMetric2 = new Metric(testMetricName2, testTags1); 78 | final String expectedMetricName2 = testMetricName2 + " dc=dc1 host=h1"; 79 | ms.addPoint(testMetric2, ts * 3, value * 3); 80 | 81 | assertTrue(ms.getSeries(Query.parse("test")).isEmpty()); 82 | assertTrue(ms.getSeries(Query.parse("test host=h1")).isEmpty()); 83 | assertTrue(ms.getSeries(Query.parse("test host=h1 dc=dc1")).isEmpty()); 84 | 85 | final Point point21 = new Point(ts * 3, value * 3); 86 | assertThat(ms.getSeries(Query.parse(testMetricName2 + queryTagString)), 87 | IsIterableContainingInOrder.contains(new TimeSeries(expectedMetricName2, 88 | Collections.singletonList(point21)))); 89 | assertThat(ms.getSeries(Query.parse(testMetricName1 + queryTagString)), 90 | IsIterableContainingInOrder.contains(timeseries12)); 91 | 92 | // Add duplicate point to metric2 93 | ms.addPoint(testMetric2, ts * 3, value * 3); 94 | List expectedPoints4 = Arrays.asList(point21, point21); 95 | assertThat(ms.getSeries(Query.parse(testMetricName2 + queryTagString)), 96 | IsIterableContainingInOrder.contains(new TimeSeries(expectedMetricName2, expectedPoints4))); 97 | assertThat(ms.getSeries(Query.parse(testMetricName1 + queryTagString)), 98 | IsIterableContainingInOrder.contains(timeseries12)); 99 | 100 | // Add third point to metric 2 101 | ms.addPoint(testMetric2, ts * 4, value * 4); 102 | List expectedPoints5 = Arrays.asList(point21, point21, new Point(ts * 4, value * 4)); 103 | assertThat(ms.getSeries(Query.parse(testMetricName2 + queryTagString)), 104 | IsIterableContainingInOrder.contains((new TimeSeries(expectedMetricName2, expectedPoints5)))); 105 | assertThat(ms.getSeries(Query.parse(testMetricName1 + queryTagString)), 106 | IsIterableContainingInOrder.contains(timeseries12)); 107 | 108 | assertTrue(ms.getSeries(Query.parse("test")).isEmpty()); 109 | assertTrue(ms.getSeries(Query.parse("test host=h1")).isEmpty()); 110 | assertTrue(ms.getSeries(Query.parse("test host=h1 dc=dc1")).isEmpty()); 111 | } 112 | 113 | 114 | public void testMultipleTimeSeriesResponse() { 115 | final String testMetricName1 = "testMetric1"; 116 | final List testTags1 = Arrays.asList("host=h1", "dc=dc1"); 117 | final List testTags2 = Arrays.asList("host=h2", "dc=dc1"); 118 | assertTrue(ms.getSeries(new Query("test", Collections.emptyList())).isEmpty()); 119 | 120 | final Metric testMetric1 = new Metric(testMetricName1, testTags1); 121 | final Metric testMetric2 = new Metric(testMetricName1, testTags2); 122 | 123 | ms.addPoint(testMetric1, ts, value); 124 | ms.addPoint(testMetric2, ts, value); 125 | Point p1 = new Point(ts, value); 126 | 127 | assertThat(ms.getSeries(Query.parse(testMetricName1 + " dc=dc1")), 128 | IsIterableContainingInOrder.contains( 129 | new TimeSeries(testMetricName1 + " dc=dc1 host=h1", Collections.singletonList(p1)), 130 | new TimeSeries(testMetricName1 + " dc=dc1 host=h2", Collections.singletonList(p1)))); 131 | } 132 | 133 | // TODO: Query corrupt tag store and metric store. 134 | } 135 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/writer/kafka/KafkaMetricWriter.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.writer.kafka; 2 | 3 | import com.pinterest.yuvi.chunk.ChunkManager; 4 | import com.pinterest.yuvi.chunk.ReadOnlyChunkInsertionException; 5 | import com.pinterest.yuvi.thrift.TextMessage; 6 | import com.pinterest.yuvi.writer.MetricWriter; 7 | 8 | import org.apache.kafka.clients.consumer.ConsumerRecord; 9 | import org.apache.kafka.clients.consumer.ConsumerRecords; 10 | import org.apache.kafka.clients.consumer.KafkaConsumer; 11 | import org.apache.kafka.common.TopicPartition; 12 | import org.slf4j.Logger; 13 | import org.slf4j.LoggerFactory; 14 | 15 | import java.util.Arrays; 16 | import java.util.List; 17 | import java.util.Properties; 18 | import java.util.concurrent.RejectedExecutionException; 19 | 20 | /** 21 | * KafkaMetricWriter ingests metrics wrapped in a TextMessage from Kafka into a Yuvi ChunkManager. 22 | * 23 | * Currently, this class ingests metrics in a thrift format since it is more efficient than the JSON 24 | * format used by OpenTSDB kafka plugin. The class is implemented in a way that the ingestion format 25 | * from kafka is pluggable, so this can be changed easily in future. 26 | * 27 | * If both a kafkaTopicName and kafkaTopicPartition are specified, we will assign the consumer to 28 | * that partition. If kafkaTopicPartition is empty, we will subscribe to the entire topic. 29 | * 30 | * TODO: Support ingesting messages in OpenTSDB JSON format from Kafka. 31 | */ 32 | public class KafkaMetricWriter implements MetricWriter { 33 | 34 | private static Logger LOG = LoggerFactory.getLogger(KafkaMetricWriter.class); 35 | 36 | // Log delayed metrics to a different logger to reduce log spam. 37 | private static Logger DELAYED_METRICS_LOG = LoggerFactory.getLogger("DELAYED_METRICS"); 38 | 39 | private final ChunkManager chunkManager; 40 | 41 | private final KafkaConsumer consumer; 42 | private final String kafkaTopic; 43 | private final int kafkaTopicPartition; 44 | private long kafkaPollTimeoutMs = 100; 45 | 46 | public KafkaMetricWriter(ChunkManager chunkManager, String kafkaTopicName, 47 | String kafkaTopicPartition, String kafkaBootStrapServers, 48 | String kafkaClientGroup, String kafkaAutoCommit, 49 | String kafkaAutoCommitInterval, String kafkaSessionTimeout) { 50 | 51 | LOG.info("Kafka params are: kafkaTopicName: {}, kafkaTopicPartition: {}, " 52 | + "kafkaBootstrapServers:{}, kafkaClientGroup: {}, kafkaAutoCommit:{}, " 53 | + "kafkaAutoCommitInterval: {}, kafkaSessionTimeout: {}", kafkaTopicName, 54 | kafkaTopicPartition, kafkaBootStrapServers, kafkaClientGroup, kafkaAutoCommit, 55 | kafkaAutoCommitInterval, kafkaSessionTimeout); 56 | 57 | if (chunkManager == null || kafkaTopicPartition == null 58 | || kafkaTopicName == null || kafkaTopicName.isEmpty() 59 | || kafkaBootStrapServers == null || kafkaBootStrapServers.isEmpty() 60 | || kafkaClientGroup == null || kafkaClientGroup.isEmpty() 61 | || kafkaAutoCommit == null || kafkaAutoCommit.isEmpty() 62 | || kafkaAutoCommitInterval == null || kafkaAutoCommitInterval.isEmpty() 63 | || kafkaSessionTimeout == null || kafkaSessionTimeout.isEmpty()) { 64 | throw new IllegalArgumentException("Kafka params can't be null or empty."); 65 | } 66 | 67 | this.chunkManager = chunkManager; 68 | 69 | // Create kafka consumer 70 | this.kafkaTopic = kafkaTopicName; 71 | 72 | int sessionTimeoutMs = new Integer(kafkaSessionTimeout.trim()); 73 | 74 | Properties props = new Properties(); 75 | props.put("bootstrap.servers", kafkaBootStrapServers); 76 | props.put("group.id", kafkaClientGroup); 77 | props.put("enable.auto.commit", kafkaAutoCommit); 78 | props.put("auto.commit.interval.ms", kafkaAutoCommitInterval); 79 | props.put("session.timeout.ms", sessionTimeoutMs); 80 | props.put("heartbeat.interval.ms", sessionTimeoutMs/3); 81 | props.put("key.deserializer", "org.apache.kafka.common.serialization.ByteArrayDeserializer"); 82 | props.put("value.deserializer", "com.pinterest.yuvi.writer.kafka.ThriftTextMessageDeserializer"); 83 | this.consumer = new KafkaConsumer<>(props); 84 | if (kafkaTopicPartition.isEmpty()) { 85 | this.kafkaTopicPartition = -1; 86 | LOG.info("Subscribing to kafka topic {}", this.kafkaTopic); 87 | consumer.subscribe(Arrays.asList(this.kafkaTopic), 88 | new MetricsConsumerRebalanceListener()); 89 | } else { 90 | this.kafkaTopicPartition = new Integer(kafkaTopicPartition); 91 | LOG.info("Assigned to kafka topic {} and partition {}", 92 | this.kafkaTopic, this.kafkaTopicPartition); 93 | consumer.assign(Arrays.asList(new TopicPartition(this.kafkaTopic, this.kafkaTopicPartition))); 94 | } 95 | } 96 | 97 | public void start() { 98 | LOG.info("Starting metrics reader."); 99 | 100 | while (true) { 101 | try { 102 | ConsumerRecords records; 103 | records = consumer.poll(kafkaPollTimeoutMs); 104 | LOG.debug("Fetched records." + records.count()); 105 | 106 | try { 107 | insertRecords(records); 108 | } catch (Exception e) { 109 | LOG.error("Error processing messages from Kafka", e); 110 | } 111 | } catch (RejectedExecutionException e) { 112 | // This case shouldn't happen since there is only one thread queuing tasks here and we check 113 | // that the queue is empty before polling kafka. 114 | LOG.error("Rejected execution shouldn't happen ", e); 115 | } catch (Exception e) { 116 | LOG.error("Unhandled exception ", e); 117 | } 118 | } 119 | } 120 | 121 | private void insertRecords(ConsumerRecords records) { 122 | int validMetrics = 0; 123 | int invalidMetrics = 0; 124 | int delayedMetrics = 0; 125 | int recordCount = 0; 126 | for (ConsumerRecord record : records) { 127 | recordCount = recordCount + 1; 128 | List metrics = record.value().getMessages(); 129 | for (String metric: metrics) { 130 | try { 131 | chunkManager.addMetric(metric); 132 | validMetrics = validMetrics + 1; 133 | } catch (ReadOnlyChunkInsertionException e) { 134 | // DELAYED_METRICS_LOG.debug("Error ingesting metric {}", metric); 135 | delayedMetrics = delayedMetrics + 1; 136 | } catch (Exception e) { 137 | LOG.error("Error ingesting metric {}", metric, e); 138 | invalidMetrics = invalidMetrics + 1; 139 | } 140 | } 141 | } 142 | LOG.info("Processed {} records with {} valid metrics, {} invalid metrics, {} delayed metrics", 143 | recordCount, validMetrics, invalidMetrics, delayedMetrics); 144 | } 145 | 146 | public void close() { 147 | // TODO: Close consumer properly. 148 | if (consumer != null) { 149 | LOG.info("Closing kafka consumer"); 150 | consumer.close(); 151 | } 152 | } 153 | } 154 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/chunk/ChunkImplTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import static com.pinterest.yuvi.chunk.MetricUtils.parseAndAddOpenTSDBMetric; 4 | import static com.pinterest.yuvi.chunk.MetricUtils.makeMetricString; 5 | import static junit.framework.TestCase.assertFalse; 6 | import static org.junit.Assert.assertEquals; 7 | import static org.junit.Assert.assertTrue; 8 | 9 | import com.pinterest.yuvi.metricandtagstore.MetricsAndTagStoreImpl; 10 | import com.pinterest.yuvi.metricstore.VarBitMetricStore; 11 | import com.pinterest.yuvi.models.Point; 12 | import com.pinterest.yuvi.models.TimeSeries; 13 | import com.pinterest.yuvi.tagstore.InvertedIndexTagStore; 14 | import com.pinterest.yuvi.tagstore.Query; 15 | 16 | import org.hamcrest.collection.IsIterableContainingInOrder; 17 | import org.junit.Assert; 18 | import org.junit.Before; 19 | import org.junit.Test; 20 | 21 | import java.util.Arrays; 22 | import java.util.Collections; 23 | import java.util.List; 24 | 25 | public class ChunkImplTest { 26 | 27 | private final long startTime = 1488499200; // Fri, 03 Mar 2017 00:00:00 UTC 28 | private final long endTime = 1488499200 + 3600 * 2; 29 | 30 | private final long testTs = 100L; 31 | private final double testValue = 10; 32 | 33 | private Chunk chunk; 34 | 35 | @Before 36 | public void setUp() { 37 | chunk = new ChunkImpl( 38 | new MetricsAndTagStoreImpl(new InvertedIndexTagStore(20, 20), new VarBitMetricStore()), 39 | new ChunkInfo("test", startTime, endTime)); 40 | } 41 | 42 | @Test 43 | public void testChunkContainsData() { 44 | assertTrue(chunk.containsDataInTimeRange(startTime - 1, endTime + 1)); 45 | assertTrue(chunk.containsDataInTimeRange(startTime + 1, endTime + 1)); 46 | assertTrue(chunk.containsDataInTimeRange(startTime + 1, endTime - 1)); 47 | assertTrue(chunk.containsDataInTimeRange(startTime - 1, endTime - 1)); 48 | 49 | assertTrue(chunk.containsDataInTimeRange(startTime, endTime)); 50 | assertTrue(chunk.containsDataInTimeRange(startTime, endTime - 1)); 51 | assertTrue(chunk.containsDataInTimeRange(startTime, endTime + 1)); 52 | assertTrue(chunk.containsDataInTimeRange(startTime + 1, endTime)); 53 | assertTrue(chunk.containsDataInTimeRange(startTime - 1, endTime)); 54 | 55 | assertFalse(chunk.containsDataInTimeRange(startTime - 10000, endTime - 10000)); 56 | assertFalse(chunk.containsDataInTimeRange(startTime + 10000, endTime + 10000)); 57 | } 58 | 59 | @Test 60 | public void testChunkIngestion() { 61 | final String testMetricName1 = "testMetric1"; 62 | 63 | // 1 metric 1 data point 64 | final String expectedMetricName1 = testMetricName1 + " dc=dc1 host=h1"; 65 | final String queryTagString = " host=h1 dc=dc1"; 66 | 67 | String inputTagString = "host=h1 dc=dc1"; 68 | parseAndAddOpenTSDBMetric(makeMetricString(testMetricName1, inputTagString, testTs, testValue), 69 | chunk); 70 | 71 | assertTrue(chunk.query(Query.parse("test")).isEmpty()); 72 | assertTrue(chunk.query(Query.parse("test host=h1")).isEmpty()); 73 | assertTrue(chunk.query(Query.parse("test host=h1 dc=dc1")).isEmpty()); 74 | 75 | final List series1 = chunk.query(Query.parse(testMetricName1 + queryTagString)); 76 | assertEquals(1, series1.size()); 77 | assertEquals(expectedMetricName1, series1.get(0).getMetric()); 78 | assertEquals(1, series1.get(0).getPoints().size()); 79 | Assert.assertThat(series1.get(0).getPoints(), 80 | IsIterableContainingInOrder.contains(new Point(testTs, testValue))); 81 | 82 | // 1 metric 2 points 83 | parseAndAddOpenTSDBMetric( 84 | makeMetricString(testMetricName1, inputTagString, testTs * 2, testValue * 2), chunk); 85 | 86 | assertTrue(chunk.query(Query.parse("test")).isEmpty()); 87 | assertTrue(chunk.query(Query.parse("test host=h1")).isEmpty()); 88 | assertTrue(chunk.query(Query.parse("test host=h1 dc=dc1")).isEmpty()); 89 | 90 | final List series2 = chunk.query(Query.parse(testMetricName1 + queryTagString)); 91 | assertEquals(1, series2.size()); 92 | assertEquals(expectedMetricName1, series2.get(0).getMetric()); 93 | assertEquals(2, series2.get(0).getPoints().size()); 94 | List expectedPoints2 = 95 | Arrays.asList(new Point(testTs, testValue), new Point(testTs * 2, testValue * 2)); 96 | final TimeSeries timeseries12 = new TimeSeries(expectedMetricName1, expectedPoints2); 97 | Assert.assertThat(series2, IsIterableContainingInOrder.contains(timeseries12)); 98 | 99 | // 2 metrics 2 points 100 | final String testMetricName2 = "testMetric2"; 101 | final String expectedMetricName2 = testMetricName2 + " dc=dc1 host=h1"; 102 | parseAndAddOpenTSDBMetric( 103 | makeMetricString(testMetricName2, inputTagString, testTs * 3, testValue * 3), chunk); 104 | 105 | assertTrue(chunk.query(Query.parse("test")).isEmpty()); 106 | assertTrue(chunk.query(Query.parse("test host=h1")).isEmpty()); 107 | assertTrue(chunk.query(Query.parse("test host=h1 dc=dc1")).isEmpty()); 108 | 109 | final Point point21 = new Point(testTs * 3, testValue * 3); 110 | Assert.assertThat(chunk.query(Query.parse(testMetricName2 + queryTagString)), 111 | IsIterableContainingInOrder.contains(new TimeSeries(expectedMetricName2, 112 | Collections.singletonList(point21)))); 113 | Assert.assertThat(chunk.query(Query.parse(testMetricName1 + queryTagString)), 114 | IsIterableContainingInOrder.contains(timeseries12)); 115 | 116 | // Add duplicate point to metric2 117 | parseAndAddOpenTSDBMetric( 118 | makeMetricString(testMetricName2, inputTagString, testTs * 3, testValue * 3), chunk); 119 | List expectedPoints4 = Arrays.asList(point21, point21); 120 | Assert.assertThat(chunk.query(Query.parse(testMetricName2 + queryTagString)), 121 | IsIterableContainingInOrder.contains(new TimeSeries(expectedMetricName2, expectedPoints4))); 122 | Assert.assertThat(chunk.query(Query.parse(testMetricName1 + queryTagString)), 123 | IsIterableContainingInOrder.contains(timeseries12)); 124 | 125 | // Add third point to metric 2 126 | parseAndAddOpenTSDBMetric( 127 | makeMetricString(testMetricName2, inputTagString, testTs * 4, testValue * 4), chunk); 128 | List 129 | expectedPoints5 = 130 | Arrays.asList(point21, point21, new Point(testTs * 4, testValue * 4)); 131 | Assert.assertThat(chunk.query(Query.parse(testMetricName2 + queryTagString)), 132 | IsIterableContainingInOrder.contains(new TimeSeries(expectedMetricName2, expectedPoints5))); 133 | Assert.assertThat(chunk.query(Query.parse(testMetricName1 + queryTagString)), 134 | IsIterableContainingInOrder.contains(timeseries12)); 135 | 136 | assertTrue(chunk.query(Query.parse("test")).isEmpty()); 137 | assertTrue(chunk.query(Query.parse("test host=h1")).isEmpty()); 138 | assertTrue(chunk.query(Query.parse("test host=h1 dc=dc1")).isEmpty()); 139 | } 140 | 141 | @Test 142 | public void testMultipleTimeSeriesResponse() { 143 | setUp(); 144 | final String testMetricName1 = "testMetric1"; 145 | final List testTags1 = Arrays.asList("host=h1", "dc=dc1"); 146 | final List testTags2 = Arrays.asList("host=h2", "dc=dc1"); 147 | assertTrue(chunk.query(new Query("test", Collections.emptyList())).isEmpty()); 148 | 149 | parseAndAddOpenTSDBMetric( 150 | makeMetricString(testMetricName1, "host=h1 dc=dc1", testTs, testValue), chunk); 151 | parseAndAddOpenTSDBMetric( 152 | makeMetricString(testMetricName1, "host=h2 dc=dc1", testTs, testValue), chunk); 153 | 154 | Point p1 = new Point(testTs, testValue); 155 | 156 | Assert.assertThat(chunk.query(Query.parse(testMetricName1 + " dc=dc1")), 157 | IsIterableContainingInOrder.contains( 158 | new TimeSeries(testMetricName1 + " dc=dc1 host=h1", Collections.singletonList(p1)), 159 | new TimeSeries(testMetricName1 + " dc=dc1 host=h2", Collections.singletonList(p1)))); 160 | } 161 | } 162 | -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/metricstore/VarBitTimeSeries.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.metricstore; 2 | 3 | import com.pinterest.yuvi.bitstream.BitStream; 4 | 5 | import com.google.common.annotations.VisibleForTesting; 6 | 7 | import java.nio.ByteBuffer; 8 | import java.util.Collections; 9 | import java.util.HashMap; 10 | import java.util.Map; 11 | 12 | /** 13 | * VarBitTimeSeries is able to compress a time series points into a binary format, using the 14 | * algorithm 15 | * described in this paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf. 16 | * 17 | * The points in the time series need not be sequential, and duplicate points are allowed. It should 18 | * be noted that the some storage efficiencies would be lost in the presense of non-sequential 19 | * inputs or in presence of duplicates. 20 | * 21 | * TODO: Merge timestamps and values into a single stream to improve data locality. 22 | * 23 | * Use the append interface to append the points and @CachingVarBitTimeSeriesIterator to read 24 | * them back. 25 | * @see CachingVarBitTimeSeriesIterator 26 | */ 27 | @VisibleForTesting 28 | public class VarBitTimeSeries { 29 | 30 | public static final long MAX_UNIX_TIMESTAMP = Integer.MAX_VALUE; 31 | public static final int BLOCK_HEADER_OFFSET_SECS = 120 * 60; // 2 hour blocks. 32 | 33 | // Picking good initial sizes for these values will reduce GC overhead. 34 | public static final short DEFAULT_TIMESTAMP_BITSTREAM_SIZE = 1; 35 | public static final short DEFAULT_VALUE_BITSTREAM_SIZE = 1; 36 | 37 | private int size; 38 | 39 | private long prevTimestamp; 40 | private long prevTimestampDelta; 41 | private final BitStream timestamps; 42 | 43 | 44 | private int previousLeadingZeros; 45 | private int previousTrailingZeros; 46 | private long previousValue; 47 | private final BitStream values; 48 | 49 | public VarBitTimeSeries() { 50 | timestamps = new BitStream(DEFAULT_TIMESTAMP_BITSTREAM_SIZE); 51 | values = new BitStream(DEFAULT_VALUE_BITSTREAM_SIZE); 52 | } 53 | 54 | /** 55 | * Append a single point to the time series. 56 | * 57 | * @param timestamp a unix timestamp in seconds 58 | * @param value a floating-point value 59 | */ 60 | public synchronized void append(long timestamp, double value) { 61 | if (timestamp < 0 || timestamp > MAX_UNIX_TIMESTAMP) { 62 | throw new IllegalArgumentException("Timestamp is not a valid unix timestamp: " + timestamp); 63 | } 64 | 65 | if (size == 0) { 66 | appendFirstPoint(timestamp, value); 67 | } else { 68 | appendNextPoint(timestamp, value); 69 | } 70 | size++; 71 | } 72 | 73 | private void appendNextPoint(long timestamp, double value) { 74 | appendTimestamp(timestamp); 75 | appendValue(value); 76 | } 77 | 78 | private void appendValue(double value) { 79 | long doubleToLongBits = Double.doubleToLongBits(value); 80 | 81 | long xorValue = doubleToLongBits ^ previousValue; 82 | 83 | int numberOfLeadingZeros = Long.numberOfLeadingZeros(xorValue); 84 | if (numberOfLeadingZeros >= 32) { 85 | numberOfLeadingZeros = 31; 86 | } 87 | 88 | int numberOfTrailingZeros = Long.numberOfTrailingZeros(xorValue); 89 | if (numberOfTrailingZeros >= 64) { 90 | numberOfTrailingZeros = 63; 91 | } 92 | 93 | if (xorValue == 0) { 94 | values.write(1, 0); 95 | } else if (numberOfLeadingZeros >= previousLeadingZeros 96 | && numberOfTrailingZeros >= previousTrailingZeros) { 97 | values.write(2, 0b10); 98 | values.write(64 - previousLeadingZeros - previousTrailingZeros, 99 | xorValue >>> previousTrailingZeros); 100 | } else { 101 | values.write(2, 0b11); 102 | values.write(5, numberOfLeadingZeros); 103 | int m = 64 - numberOfLeadingZeros - numberOfTrailingZeros; 104 | values.write(6, numberOfTrailingZeros); 105 | values.write(m, xorValue >>> numberOfTrailingZeros); 106 | previousTrailingZeros = numberOfTrailingZeros; 107 | previousLeadingZeros = numberOfLeadingZeros; 108 | } 109 | previousValue = doubleToLongBits; 110 | } 111 | 112 | private void appendTimestamp(long timestamp) { 113 | long delta = timestamp - prevTimestamp; 114 | long deltaOfDelta = delta - prevTimestampDelta; 115 | if (deltaOfDelta == 0) { 116 | timestamps.write(1, 0); 117 | } else if (deltaOfDelta >= -63 && deltaOfDelta <= 64) { 118 | timestamps.write(9, (deltaOfDelta + 63) | 0b100000000); 119 | } else if (deltaOfDelta >= -255 && deltaOfDelta <= 256) { 120 | timestamps.write(12, deltaOfDelta + 255 | 0b110000000000); 121 | } else if (deltaOfDelta >= -2047 && deltaOfDelta <= 2048) { 122 | timestamps.write(16, deltaOfDelta + 2047 | 0b1110000000000000); 123 | } else { 124 | timestamps.write(4, 0b1111); 125 | /** 126 | * There is a bug in gorilla algorithm where the delta of delta difference can't be encoded in 127 | * 32 bits. Adding such a value corrupts the time series encoding. However, this only happens 128 | * when a timestamp from 1970 is encoded followed by a value from 2016. Since the farthest 129 | * timestamps in a chunk will be apart from each other by a few hours, this case will never 130 | * happen. Instead of fixing the encoding and hurt the compression ratio, we throw an 131 | * exception in this case and reject the data point. Please refer to the test case for an 132 | * example. 133 | */ 134 | if ((Math.log(deltaOfDelta + 2147483647) / Math.log(2)) > 32) { 135 | throw new IllegalArgumentException("timestamp overflows 32 bits: " + timestamp); 136 | } 137 | timestamps.write(32, deltaOfDelta + 2147483647); 138 | } 139 | prevTimestamp = timestamp; 140 | prevTimestampDelta = delta; 141 | } 142 | 143 | /** 144 | * Starting timestamp is at the 2 hour starting window of the current timestamp. The first 145 | * timestamp is encoded as a delta from the starting timestamp in 14 bits. 146 | */ 147 | private void appendFirstPoint(long timestamp, double value) { 148 | long twoHourTimestampOverage = timestamp % BLOCK_HEADER_OFFSET_SECS; 149 | long blockHeaderTimestamp = timestamp - twoHourTimestampOverage; 150 | timestamps.write(32, blockHeaderTimestamp); 151 | prevTimestamp = timestamp; 152 | prevTimestampDelta = prevTimestamp - blockHeaderTimestamp; 153 | timestamps.write(14, prevTimestampDelta); 154 | 155 | // Store first value with no compression. 156 | long longValue = Double.doubleToLongBits(value); 157 | values.write(64, longValue); 158 | previousValue = longValue; 159 | previousLeadingZeros = 64; 160 | previousTrailingZeros = 64; 161 | } 162 | 163 | /** 164 | * Read a snapshot of the time series data that has been written. 165 | * @return an object that can deserialize the compressed data. 166 | */ 167 | public synchronized TimeSeriesIterator read() { 168 | //TODO: The read object returns the values at a point instead of returning all values when 169 | // called. Change it. 170 | return new CachingVarBitTimeSeriesIterator(size, timestamps.read(), values.read()); 171 | } 172 | 173 | @VisibleForTesting 174 | int getSize() { 175 | return size; 176 | } 177 | 178 | public Map getStats() { 179 | Map stats = new HashMap<>(); 180 | stats.put("pointsCount", new Double(size)); 181 | timestamps.getStats().entrySet().forEach( 182 | entry -> stats.put("timestamps_" + entry.getKey(), entry.getValue())); 183 | values.getStats().entrySet().forEach( 184 | entry -> stats.put("values_" + entry.getKey(), entry.getValue())); 185 | return Collections.unmodifiableMap(stats); 186 | } 187 | 188 | @VisibleForTesting 189 | BitStream getTimestamps() { 190 | return timestamps; 191 | } 192 | 193 | @VisibleForTesting 194 | BitStream getValues() { 195 | return values; 196 | } 197 | 198 | public int getSerializedByteSize() { 199 | return Integer.BYTES // Size of counter 200 | + timestamps.getSerializedByteSize() // Size of timestamps bit stream 201 | + values.getSerializedByteSize(); // Size of values 202 | } 203 | 204 | public void serialize(ByteBuffer buffer) throws Exception { 205 | buffer.putInt(size); 206 | timestamps.serialize(buffer); 207 | values.serialize(buffer); 208 | } 209 | 210 | public static TimeSeriesIterator deserialize(final ByteBuffer buffer) { 211 | try { 212 | int size = buffer.getInt(); 213 | BitStream timestamps = BitStream.deserialize(buffer); 214 | BitStream values = BitStream.deserialize(buffer); 215 | return new CachingVarBitTimeSeriesIterator(size, timestamps.read(), values.read()); 216 | } catch (Exception e) { 217 | return null; 218 | } 219 | } 220 | } 221 | -------------------------------------------------------------------------------- /yuvi/pom.xml: -------------------------------------------------------------------------------- 1 | 4 | 4.0.0 5 | 6 | com.pinterest.yuvi 7 | yuvi 8 | 0.1-SNAPSHOT 9 | jar 10 | 11 | 12 | UTF-8 13 | 1.19 14 | 1.8 15 | yuvi 16 | 17 | 18 | 19 | 20 | Apache License, Version 2.0 21 | https://www.apache.org/licenses/LICENSE-2.0.txt 22 | repo 23 | A business-friendly OSS license 24 | 25 | 26 | 27 | 28 | 29 | org.roaringbitmap 30 | RoaringBitmap 31 | 0.6.27 32 | 33 | 34 | net.openhft 35 | chronicle-map 36 | 3.13.0 37 | 38 | 39 | org.apache.commons 40 | commons-lang3 41 | 3.6 42 | 43 | 44 | 45 | org.slf4j 46 | slf4j-api 47 | 1.7.7 48 | 49 | 50 | 51 | 52 | org.apache.kafka 53 | kafka-clients 54 | 0.11.0.1 55 | 56 | 57 | org.apache.thrift 58 | libthrift 59 | 0.10.0 60 | 61 | 62 | 63 | 64 | 65 | com.google.guava 66 | guava 67 | 14.0 68 | 69 | 70 | 71 | 72 | junit 73 | junit 74 | 4.11 75 | 76 | 77 | org.hamcrest 78 | hamcrest-all 79 | 1.3 80 | 81 | 82 | 83 | 84 | 85 | 86 | org.apache.maven.plugins 87 | maven-compiler-plugin 88 | 3.1 89 | 90 | ${javac.target} 91 | ${javac.target} 92 | ${javac.target} 93 | ${javac.target} 94 | -Xlint:unchecked 95 | 96 | 97 | 98 | org.apache.maven.plugins 99 | maven-shade-plugin 100 | 2.4.1 101 | 102 | 103 | package 104 | 105 | shade 106 | 107 | 108 | ${uberjar.name} 109 | 110 | 112 | 113 | 114 | 115 | 116 | 120 | *:* 121 | 122 | META-INF/*.SF 123 | META-INF/*.DSA 124 | META-INF/*.RSA 125 | 126 | 127 | 128 | 129 | 130 | 131 | 132 | 133 | 134 | org.apache.thrift.tools 135 | maven-thrift-plugin 136 | 0.1.11 137 | 138 | java 139 | /usr/local/bin/thrift 140 | 141 | 142 | 143 | thrift-sources 144 | generate-sources 145 | 146 | compile 147 | 148 | 149 | 150 | thrift-test-sources 151 | generate-test-sources 152 | 153 | testCompile 154 | 155 | 156 | 157 | 158 | 159 | 160 | 161 | 162 | maven-clean-plugin 163 | 2.5 164 | 165 | 166 | maven-deploy-plugin 167 | 2.8.1 168 | 169 | 170 | maven-install-plugin 171 | 2.5.1 172 | 173 | 174 | maven-jar-plugin 175 | 2.4 176 | 177 | 178 | maven-javadoc-plugin 179 | 2.9.1 180 | 181 | 182 | maven-resources-plugin 183 | 2.6 184 | 185 | 186 | maven-site-plugin 187 | 3.3 188 | 189 | 190 | maven-source-plugin 191 | 2.2.1 192 | 193 | 194 | maven-surefire-plugin 195 | 2.17 196 | 197 | 198 | org.apache.maven.plugins 199 | maven-compiler-plugin 200 | 3.5.1 201 | 202 | 1.8 203 | 1.8 204 | 205 | 206 | 207 | 208 | 209 | 210 | -------------------------------------------------------------------------------- /yuvi/src/test/java/com/pinterest/yuvi/bitstream/BitStreamTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.bitstream; 2 | 3 | import static org.junit.Assert.assertEquals; 4 | import static org.junit.Assert.assertTrue; 5 | 6 | import com.pinterest.yuvi.bitstream.BitStream; 7 | import com.pinterest.yuvi.bitstream.BitStreamIterator; 8 | 9 | import org.junit.Rule; 10 | import org.junit.Test; 11 | import org.junit.rules.ExpectedException; 12 | 13 | import java.nio.ByteBuffer; 14 | 15 | public class BitStreamTest { 16 | 17 | @Rule 18 | public ExpectedException thrown = ExpectedException.none(); 19 | 20 | @Test 21 | public void testEncode() { 22 | BitStream stream = new BitStream(1); 23 | stream.write(10, 10); 24 | stream.write(4, 4); 25 | assertEquals(1, stream.getCapacity()); 26 | assertEquals(0, stream.getIndex()); 27 | assertEquals(14, stream.getShift()); 28 | assertEquals(1, stream.getData().length); 29 | assertEquals(stream.getCapacity(), stream.getData().length); 30 | 31 | BitStreamIterator it = stream.read(); 32 | assertTrue(it.tryRead(10, 10)); 33 | assertTrue(it.tryRead(4, 4)); 34 | 35 | stream.write(63, 40); 36 | assertEquals(2, stream.getCapacity()); 37 | assertEquals(1, stream.getIndex()); 38 | assertEquals(13, stream.getShift()); 39 | assertEquals(2, stream.getData().length); 40 | assertEquals(stream.getCapacity(), stream.getData().length); 41 | 42 | BitStreamIterator it2 = stream.read(); 43 | assertTrue(it2.tryRead(10, 10)); 44 | assertTrue(it2.tryRead(4, 4)); 45 | assertTrue(it2.tryRead(63, 40)); 46 | 47 | stream.write(62, 60); 48 | assertEquals(4, stream.getCapacity()); 49 | assertEquals(2, stream.getIndex()); 50 | assertEquals(11, stream.getShift()); 51 | assertEquals(4, stream.getData().length); 52 | assertEquals(stream.getCapacity(), stream.getData().length); 53 | 54 | BitStreamIterator it3 = stream.read(); 55 | assertTrue(it3.tryRead(10, 10)); 56 | assertTrue(it3.tryRead(4, 4)); 57 | assertTrue(it3.tryRead(63, 40)); 58 | assertTrue(it3.tryRead(62, 60)); 59 | } 60 | 61 | @Test 62 | public void testDefaultInitialization() { 63 | BitStream stream = new BitStream(); 64 | assertEquals(BitStream.DEFAULT_INITIAL_CAPACITY, stream.getCapacity()); 65 | assertEquals(0, stream.getIndex()); 66 | assertEquals(0, stream.getShift()); 67 | assertEquals(16, stream.getData().length); 68 | assertEquals(16, stream.getCapacity()); 69 | assertEquals(stream.getCapacity(), stream.getData().length); 70 | } 71 | 72 | @Test 73 | public void testSerializeDeserialize() throws Exception { 74 | BitStream stream = new BitStream( 2); 75 | stream.write(10, 10); 76 | stream.write(63, 4); 77 | stream.write(60, 61); 78 | assertEquals(4, stream.getCapacity()); 79 | assertEquals(2, stream.getIndex()); 80 | assertEquals(5, stream.getShift()); 81 | assertEquals(4, stream.getData().length); 82 | assertEquals(stream.getCapacity(), stream.getData().length); 83 | assertEquals(29, stream.getSerializedByteSize()); 84 | 85 | ByteBuffer buffer = ByteBuffer.allocate(stream.getSerializedByteSize()); 86 | stream.serialize(buffer); 87 | buffer.flip(); 88 | 89 | BitStream deserializedStream = BitStream.deserialize(buffer); 90 | BitStreamIterator it = deserializedStream.read(); 91 | assertEquals(3, deserializedStream.getCapacity()); 92 | assertEquals(2, deserializedStream.getIndex()); 93 | assertEquals(5, deserializedStream.getShift()); 94 | assertEquals(3, deserializedStream.getData().length); 95 | assertEquals(deserializedStream.getCapacity(), deserializedStream.getData().length); 96 | assertEquals(29, deserializedStream.getSerializedByteSize()); 97 | assertTrue(it.tryRead(10, 10)); 98 | assertTrue(it.tryRead(63, 4)); 99 | assertTrue(it.tryRead(60, 61)); 100 | 101 | // Write to deserialized stream and ensure that the resultant stream is a valid BitStream. 102 | deserializedStream.write(2, 1); 103 | assertEquals(29, deserializedStream.getSerializedByteSize()); 104 | 105 | ByteBuffer buffer2 = ByteBuffer.allocate(deserializedStream.getSerializedByteSize()); 106 | deserializedStream.serialize(buffer2); 107 | buffer2.flip(); 108 | 109 | BitStream deserializedStream2 = BitStream.deserialize(buffer2); 110 | assertEquals(3, deserializedStream2.getCapacity()); 111 | assertEquals(2, deserializedStream2.getIndex()); 112 | assertEquals(7, deserializedStream2.getShift()); 113 | assertEquals(3, deserializedStream2.getData().length); 114 | assertEquals(deserializedStream2.getCapacity(), deserializedStream2.getData().length); 115 | BitStreamIterator it2 = deserializedStream2.read(); 116 | assertTrue(it2.tryRead(10, 10)); 117 | assertTrue(it2.tryRead(63, 4)); 118 | assertTrue(it2.tryRead(60, 61)); 119 | assertTrue(it2.tryRead(2, 1)); 120 | 121 | // Write a long value so we can test capacity doubling. 122 | deserializedStream2.write(63, 65); 123 | assertEquals(6, deserializedStream2.getCapacity()); 124 | assertEquals(3, deserializedStream2.getIndex()); 125 | assertEquals(6, deserializedStream2.getShift()); 126 | assertEquals(6, deserializedStream2.getData().length); 127 | assertEquals(deserializedStream2.getCapacity(), deserializedStream2.getData().length); 128 | assertEquals(37, deserializedStream2.getSerializedByteSize()); 129 | 130 | ByteBuffer buffer3 = ByteBuffer.allocate(deserializedStream2.getSerializedByteSize()); 131 | deserializedStream2.serialize(buffer3); 132 | buffer3.flip(); 133 | 134 | BitStream deserializedStream3 = BitStream.deserialize(buffer3); 135 | assertEquals(4, deserializedStream3.getCapacity()); 136 | assertEquals(3, deserializedStream3.getIndex()); 137 | assertEquals(6, deserializedStream3.getShift()); 138 | assertEquals(4, deserializedStream3.getData().length); 139 | assertEquals(deserializedStream3.getCapacity(), deserializedStream3.getData().length); 140 | assertEquals(37, deserializedStream3.getSerializedByteSize()); 141 | BitStreamIterator it3 = deserializedStream3.read(); 142 | assertTrue(it3.tryRead(10, 10)); 143 | assertTrue(it3.tryRead(63, 4)); 144 | assertTrue(it3.tryRead(60, 61)); 145 | assertTrue(it3.tryRead(2, 1)); 146 | assertTrue(it3.tryRead(63, 65)); 147 | } 148 | 149 | @Test 150 | public void testSerializeDeserialzeWithLazyDataAllocation() throws Exception { 151 | BitStream stream = new BitStream(2); 152 | stream.write(1, 0); 153 | stream.write(63, 10); 154 | stream.write(64, 20); 155 | assertEquals(2, stream.getCapacity()); 156 | assertEquals(2, stream.getIndex()); // Index is 2, even though there is no data[2] 157 | assertEquals(0, stream.getShift()); 158 | assertEquals(2, stream.getData().length); 159 | assertEquals(stream.getCapacity(), stream.getData().length); 160 | assertEquals(21, stream.getSerializedByteSize()); 161 | 162 | ByteBuffer buffer = ByteBuffer.allocate(stream.getSerializedByteSize()); 163 | stream.serialize(buffer); 164 | buffer.flip(); 165 | 166 | BitStream deserializedStream = BitStream.deserialize(buffer); 167 | BitStreamIterator it = deserializedStream.read(); 168 | assertEquals(2, deserializedStream.getCapacity()); 169 | assertEquals(2, deserializedStream.getIndex()); 170 | assertEquals(0, deserializedStream.getShift()); 171 | assertEquals(2, deserializedStream.getData().length); 172 | assertEquals(deserializedStream.getCapacity(), deserializedStream.getData().length); 173 | assertEquals(21, deserializedStream.getSerializedByteSize()); 174 | assertTrue(it.tryRead(1, 0)); 175 | assertTrue(it.tryRead(63, 10)); 176 | assertTrue(it.tryRead(64, 20)); 177 | 178 | deserializedStream.write(1, 1); 179 | BitStreamIterator it1 = deserializedStream.read(); 180 | assertTrue(it1.tryRead(1, 0)); 181 | assertTrue(it1.tryRead(63, 10)); 182 | assertTrue(it1.tryRead(64, 20)); 183 | assertTrue(it1.tryRead(1, 1)); 184 | assertEquals(29, deserializedStream.getSerializedByteSize()); 185 | 186 | ByteBuffer buffer2 = ByteBuffer.allocate(deserializedStream.getSerializedByteSize()); 187 | deserializedStream.serialize(buffer2); 188 | buffer2.flip(); 189 | 190 | BitStream deserializedStream2 = BitStream.deserialize(buffer2); 191 | assertEquals(3, deserializedStream2.getCapacity()); 192 | assertEquals(2, deserializedStream2.getIndex()); 193 | assertEquals(1, deserializedStream2.getShift()); 194 | assertEquals(3, deserializedStream2.getData().length); 195 | assertEquals(deserializedStream2.getCapacity(), deserializedStream2.getData().length); 196 | BitStreamIterator it2 = deserializedStream2.read(); 197 | assertTrue(it2.tryRead(1, 0)); 198 | assertTrue(it2.tryRead(63, 10)); 199 | assertTrue(it2.tryRead(64, 20)); 200 | assertTrue(it2.tryRead(1, 1)); 201 | } 202 | 203 | @Test 204 | public void testNegativeBitEncoding() { 205 | thrown.expect(IllegalArgumentException.class); 206 | new BitStream( 2).write(-1, 10); 207 | } 208 | 209 | @Test 210 | public void testZeroBitEncoding() { 211 | thrown.expect(IllegalArgumentException.class); 212 | new BitStream( 2).write(0, 10); 213 | } 214 | 215 | @Test 216 | public void test9ByteEncoding() { 217 | thrown.expect(IllegalArgumentException.class); 218 | new BitStream( 2).write(65, 10); 219 | } 220 | } 221 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | 2 | Apache License 3 | Version 2.0, January 2004 4 | http://www.apache.org/licenses/ 5 | 6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 7 | 8 | 1. Definitions. 9 | 10 | "License" shall mean the terms and conditions for use, reproduction, 11 | and distribution as defined by Sections 1 through 9 of this document. 12 | 13 | "Licensor" shall mean the copyright owner or entity authorized by 14 | the copyright owner that is granting the License. 15 | 16 | "Legal Entity" shall mean the union of the acting entity and all 17 | other entities that control, are controlled by, or are under common 18 | control with that entity. For the purposes of this definition, 19 | "control" means (i) the power, direct or indirect, to cause the 20 | direction or management of such entity, whether by contract or 21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 22 | outstanding shares, or (iii) beneficial ownership of such entity. 23 | 24 | "You" (or "Your") shall mean an individual or Legal Entity 25 | exercising permissions granted by this License. 26 | 27 | "Source" form shall mean the preferred form for making modifications, 28 | including but not limited to software source code, documentation 29 | source, and configuration files. 30 | 31 | "Object" form shall mean any form resulting from mechanical 32 | transformation or translation of a Source form, including but 33 | not limited to compiled object code, generated documentation, 34 | and conversions to other media types. 35 | 36 | "Work" shall mean the work of authorship, whether in Source or 37 | Object form, made available under the License, as indicated by a 38 | copyright notice that is included in or attached to the work 39 | (an example is provided in the Appendix below). 40 | 41 | "Derivative Works" shall mean any work, whether in Source or Object 42 | form, that is based on (or derived from) the Work and for which the 43 | editorial revisions, annotations, elaborations, or other modifications 44 | represent, as a whole, an original work of authorship. For the purposes 45 | of this License, Derivative Works shall not include works that remain 46 | separable from, or merely link (or bind by name) to the interfaces of, 47 | the Work and Derivative Works thereof. 48 | 49 | "Contribution" shall mean any work of authorship, including 50 | the original version of the Work and any modifications or additions 51 | to that Work or Derivative Works thereof, that is intentionally 52 | submitted to Licensor for inclusion in the Work by the copyright owner 53 | or by an individual or Legal Entity authorized to submit on behalf of 54 | the copyright owner. For the purposes of this definition, "submitted" 55 | means any form of electronic, verbal, or written communication sent 56 | to the Licensor or its representatives, including but not limited to 57 | communication on electronic mailing lists, source code control systems, 58 | and issue tracking systems that are managed by, or on behalf of, the 59 | Licensor for the purpose of discussing and improving the Work, but 60 | excluding communication that is conspicuously marked or otherwise 61 | designated in writing by the copyright owner as "Not a Contribution." 62 | 63 | "Contributor" shall mean Licensor and any individual or Legal Entity 64 | on behalf of whom a Contribution has been received by Licensor and 65 | subsequently incorporated within the Work. 66 | 67 | 2. Grant of Copyright License. Subject to the terms and conditions of 68 | this License, each Contributor hereby grants to You a perpetual, 69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 70 | copyright license to reproduce, prepare Derivative Works of, 71 | publicly display, publicly perform, sublicense, and distribute the 72 | Work and such Derivative Works in Source or Object form. 73 | 74 | 3. Grant of Patent License. Subject to the terms and conditions of 75 | this License, each Contributor hereby grants to You a perpetual, 76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 77 | (except as stated in this section) patent license to make, have made, 78 | use, offer to sell, sell, import, and otherwise transfer the Work, 79 | where such license applies only to those patent claims licensable 80 | by such Contributor that are necessarily infringed by their 81 | Contribution(s) alone or by combination of their Contribution(s) 82 | with the Work to which such Contribution(s) was submitted. If You 83 | institute patent litigation against any entity (including a 84 | cross-claim or counterclaim in a lawsuit) alleging that the Work 85 | or a Contribution incorporated within the Work constitutes direct 86 | or contributory patent infringement, then any patent licenses 87 | granted to You under this License for that Work shall terminate 88 | as of the date such litigation is filed. 89 | 90 | 4. Redistribution. You may reproduce and distribute copies of the 91 | Work or Derivative Works thereof in any medium, with or without 92 | modifications, and in Source or Object form, provided that You 93 | meet the following conditions: 94 | 95 | (a) You must give any other recipients of the Work or 96 | Derivative Works a copy of this License; and 97 | 98 | (b) You must cause any modified files to carry prominent notices 99 | stating that You changed the files; and 100 | 101 | (c) You must retain, in the Source form of any Derivative Works 102 | that You distribute, all copyright, patent, trademark, and 103 | attribution notices from the Source form of the Work, 104 | excluding those notices that do not pertain to any part of 105 | the Derivative Works; and 106 | 107 | (d) If the Work includes a "NOTICE" text file as part of its 108 | distribution, then any Derivative Works that You distribute must 109 | include a readable copy of the attribution notices contained 110 | within such NOTICE file, excluding those notices that do not 111 | pertain to any part of the Derivative Works, in at least one 112 | of the following places: within a NOTICE text file distributed 113 | as part of the Derivative Works; within the Source form or 114 | documentation, if provided along with the Derivative Works; or, 115 | within a display generated by the Derivative Works, if and 116 | wherever such third-party notices normally appear. The contents 117 | of the NOTICE file are for informational purposes only and 118 | do not modify the License. You may add Your own attribution 119 | notices within Derivative Works that You distribute, alongside 120 | or as an addendum to the NOTICE text from the Work, provided 121 | that such additional attribution notices cannot be construed 122 | as modifying the License. 123 | 124 | You may add Your own copyright statement to Your modifications and 125 | may provide additional or different license terms and conditions 126 | for use, reproduction, or distribution of Your modifications, or 127 | for any such Derivative Works as a whole, provided Your use, 128 | reproduction, and distribution of the Work otherwise complies with 129 | the conditions stated in this License. 130 | 131 | 5. Submission of Contributions. Unless You explicitly state otherwise, 132 | any Contribution intentionally submitted for inclusion in the Work 133 | by You to the Licensor shall be under the terms and conditions of 134 | this License, without any additional terms or conditions. 135 | Notwithstanding the above, nothing herein shall supersede or modify 136 | the terms of any separate license agreement you may have executed 137 | with Licensor regarding such Contributions. 138 | 139 | 6. Trademarks. This License does not grant permission to use the trade 140 | names, trademarks, service marks, or product names of the Licensor, 141 | except as required for reasonable and customary use in describing the 142 | origin of the Work and reproducing the content of the NOTICE file. 143 | 144 | 7. Disclaimer of Warranty. Unless required by applicable law or 145 | agreed to in writing, Licensor provides the Work (and each 146 | Contributor provides its Contributions) on an "AS IS" BASIS, 147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 148 | implied, including, without limitation, any warranties or conditions 149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 150 | PARTICULAR PURPOSE. You are solely responsible for determining the 151 | appropriateness of using or redistributing the Work and assume any 152 | risks associated with Your exercise of permissions under this License. 153 | 154 | 8. Limitation of Liability. In no event and under no legal theory, 155 | whether in tort (including negligence), contract, or otherwise, 156 | unless required by applicable law (such as deliberate and grossly 157 | negligent acts) or agreed to in writing, shall any Contributor be 158 | liable to You for damages, including any direct, indirect, special, 159 | incidental, or consequential damages of any character arising as a 160 | result of this License or out of the use or inability to use the 161 | Work (including but not limited to damages for loss of goodwill, 162 | work stoppage, computer failure or malfunction, or any and all 163 | other commercial damages or losses), even if such Contributor 164 | has been advised of the possibility of such damages. 165 | 166 | 9. Accepting Warranty or Additional Liability. While redistributing 167 | the Work or Derivative Works thereof, You may choose to offer, 168 | and charge a fee for, acceptance of support, warranty, indemnity, 169 | or other liability obligations and/or rights consistent with this 170 | License. However, in accepting such obligations, You may act only 171 | on Your own behalf and on Your sole responsibility, not on behalf 172 | of any other Contributor, and only if You agree to indemnify, 173 | defend, and hold each Contributor harmless for any liability 174 | incurred by, or claims asserted against, such Contributor by reason 175 | of your accepting any such warranty or additional liability. 176 | 177 | END OF TERMS AND CONDITIONS 178 | 179 | APPENDIX: How to apply the Apache License to your work. 180 | 181 | To apply the Apache License to your work, attach the following 182 | boilerplate notice, with the fields enclosed by brackets "[]" 183 | replaced with your own identifying information. (Don't include 184 | the brackets!) The text should be enclosed in the appropriate 185 | comment syntax for the file format. We also recommend that a 186 | file or class name and description of purpose be included on the 187 | same "printed page" as the copyright notice for easier 188 | identification within third-party archives. 189 | 190 | Copyright [yyyy] [name of copyright owner] 191 | 192 | Licensed under the Apache License, Version 2.0 (the "License"); 193 | you may not use this file except in compliance with the License. 194 | You may obtain a copy of the License at 195 | 196 | http://www.apache.org/licenses/LICENSE-2.0 197 | 198 | Unless required by applicable law or agreed to in writing, software 199 | distributed under the License is distributed on an "AS IS" BASIS, 200 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 201 | See the License for the specific language governing permissions and 202 | limitations under the License. -------------------------------------------------------------------------------- /yuvi/src/main/java/com/pinterest/yuvi/chunk/ChunkManager.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.yuvi.chunk; 2 | 3 | import static java.util.stream.Collectors.groupingBy; 4 | import static java.util.stream.Collectors.mapping; 5 | import static java.util.stream.Collectors.toList; 6 | 7 | import com.pinterest.yuvi.metricandtagstore.MetricAndTagStore; 8 | import com.pinterest.yuvi.metricandtagstore.MetricsAndTagStoreImpl; 9 | import com.pinterest.yuvi.metricstore.MetricStore; 10 | import com.pinterest.yuvi.metricstore.VarBitMetricStore; 11 | import com.pinterest.yuvi.metricstore.offheap.OffHeapVarBitMetricStore; 12 | import com.pinterest.yuvi.models.Point; 13 | import com.pinterest.yuvi.models.Points; 14 | import com.pinterest.yuvi.models.TimeSeries; 15 | import com.pinterest.yuvi.tagstore.InvertedIndexTagStore; 16 | import com.pinterest.yuvi.tagstore.Metric; 17 | import com.pinterest.yuvi.tagstore.Query; 18 | import com.pinterest.yuvi.tagstore.TagStore; 19 | 20 | import com.google.common.annotations.VisibleForTesting; 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | import java.time.Duration; 25 | import java.time.Instant; 26 | import java.util.AbstractMap; 27 | import java.util.Arrays; 28 | import java.util.List; 29 | import java.util.Map; 30 | import java.util.concurrent.ConcurrentHashMap; 31 | import java.util.stream.Collectors; 32 | 33 | /** 34 | * Each instance of yuvi stores the last N hours of time series data. That data is broken down 35 | * into 2 hours worth of time series data called a chunk. So, if the ChunkManager stores a day worth 36 | * of data there will be 12 chunks in a yuvi instance. The ChunkManager is a class that manages 37 | * the chunks in a yuvi instance. 38 | */ 39 | @SuppressWarnings("unchecked") 40 | public class ChunkManager { 41 | 42 | private static final Logger LOG = LoggerFactory.getLogger(ChunkManager.class); 43 | 44 | public static Duration DEFAULT_CHUNK_DURATION = Duration.ofMinutes(120); // 2 hours. 45 | 46 | private Object chunkMapSync = new Object(); 47 | 48 | private final String dataDirectory; 49 | 50 | /** 51 | * Each chunk contains 2 hours worth of data. The chunk map is a map whose key is start time of a 52 | * 2 hours timestamp and value is a chunk for those 2 hours. 53 | */ 54 | private final Map chunkMap; 55 | 56 | private final String chunkDataPrefix; 57 | 58 | private final TagStore tagStore; 59 | 60 | public ChunkManager(String chunkDataPrefix, int expectedTagStoreSize) { 61 | this(chunkDataPrefix, expectedTagStoreSize, ""); 62 | } 63 | 64 | public ChunkManager(String chunkDataPrefix, int expectedTagStoreSize, String dataDirectory) { 65 | this.dataDirectory = dataDirectory; 66 | chunkMap = new ConcurrentHashMap<>(); 67 | this.chunkDataPrefix = chunkDataPrefix; 68 | this.tagStore = 69 | new InvertedIndexTagStore(expectedTagStoreSize, expectedTagStoreSize, dataDirectory); 70 | LOG.info("Created a chunk manager with prefix {}, initial tag store size {} and dataDirectory {}", 71 | chunkDataPrefix, expectedTagStoreSize, dataDirectory); 72 | } 73 | 74 | private Chunk makeChunk(long startTime) { 75 | Instant endTime = Instant.ofEpochSecond(startTime) 76 | .plusMillis(DEFAULT_CHUNK_DURATION.toMillis()); 77 | 78 | return new ChunkImpl( 79 | new MetricsAndTagStoreImpl(tagStore, new VarBitMetricStore()), 80 | new ChunkInfo(chunkDataPrefix + "_" + startTime, startTime, endTime.getEpochSecond())); 81 | } 82 | 83 | /** 84 | * Get or create a chunk for a specific timestamp from chunkMap. 85 | */ 86 | public Chunk getChunk(long timestamp) { 87 | // Check current chunk range. 88 | long twoHourTimestampOverage = timestamp % DEFAULT_CHUNK_DURATION.getSeconds(); 89 | long blockHeaderTimestamp = timestamp - twoHourTimestampOverage; 90 | if (chunkMap.containsKey(blockHeaderTimestamp)) { 91 | return chunkMap.get(blockHeaderTimestamp); 92 | } else { 93 | // Since multiple points may be inserted at the same time and map updated sync here. 94 | synchronized (chunkMapSync) { 95 | Chunk newChunk = makeChunk(blockHeaderTimestamp); 96 | Chunk prevChunk = chunkMap.putIfAbsent(blockHeaderTimestamp, newChunk); 97 | if (prevChunk == null) { 98 | return newChunk; 99 | } else { 100 | // Because of the synchronized lock only one thread can create a chunk once. 101 | LOG.warn("Possible race condition in Chunk Manager."); 102 | return prevChunk; 103 | } 104 | } 105 | } 106 | } 107 | 108 | /** 109 | * Parse and ingest metric string. The input metric string is in the following format. 110 | * Sample msg: put tc.proc.net.compressed.jenkins-worker-mp 1465530393 0 iface=eth0 direction=in 111 | * 112 | * The logic to parse the opentsdb metric string is added inline here intentionally. We can add 113 | * a separate parse method, but since Java can't return multiple arguments we have to create an 114 | * additional object to return these fields for a short time. This method is hot since it is 115 | * called for every metric added. Instead of adding additional garbage, inlining the logic here 116 | * for now. This is ok for now since the metrics are only in one format for now. If we need to 117 | * handle metrics in multiple formats in future, we can make this logic more pluggable. 118 | */ 119 | public void addMetric(final String metricString) { 120 | try { 121 | String[] metricParts = metricString.split(" "); 122 | if (metricParts.length > 1 && metricParts[0].equals("put")) { 123 | String metricName = metricParts[1].trim(); 124 | List rawTags = Arrays.asList(metricParts).subList(4, metricParts.length); 125 | Metric metric = new Metric(metricName, rawTags); 126 | long ts = Long.parseLong(metricParts[2].trim()); 127 | double value = Double.parseDouble(metricParts[3].trim()); 128 | 129 | Chunk chunk = getChunk(ts); 130 | if (!chunk.isReadOnly()) { 131 | chunk.addPoint(metric, ts, value); 132 | } else { 133 | throw new ReadOnlyChunkInsertionException("Inserting metric into a read only store:" 134 | + metricString); 135 | } 136 | } else { 137 | throw new IllegalArgumentException("Metric doesn't start with a put: " + metricString); 138 | } 139 | } catch (ReadOnlyChunkInsertionException re) { 140 | throw re; // Rethrow this exception since it is useful for detecting delayed metrics. 141 | } catch (Exception e) { 142 | LOG.error("metric failed with exception: ", e); 143 | throw new IllegalArgumentException("Invalid metric string " + metricString, e); 144 | } 145 | } 146 | 147 | /** 148 | * Query multiple chunks that contain data between startTs and endTs, merge their results. Return 149 | * a single list of time series events. This query assumes that the startTs and endTs align with 150 | * 2 hour boundaries. Otherwise, we return data that aligns at 2 hour boundaries. 151 | * 152 | * Currently, the underlying time series store has to decode the entire time series to get the 153 | * values at the end of the series. So, it's better to return the entire range. Statsboard and 154 | * other UIs query for data at hour boundary any ways. 155 | * 156 | * Optimizations to be considered 157 | * Query chunks in parallel. 158 | * Minimize the number of intermediate objects created. 159 | * Simplify logic using other collectors? 160 | * Optimize chunk search. Currently, we search through 12 chunk every time. 161 | * Only query the chunk for time range for the query. 162 | */ 163 | public List queryAroundChunkBoundaries(Query query, long startTsSecs, 164 | long endTsSecs, 165 | QueryAggregation queryAggregation) { 166 | 167 | // Select relavent chunks 168 | List chunksContainingData = chunkMap.values().stream() 169 | .filter(chunk -> chunk.containsDataInTimeRange(startTsSecs, endTsSecs)) 170 | .collect(toList()); 171 | 172 | // TODO: Replace parallelStream with a dedicated thread pool. 173 | // Query the chunks in parallel. 174 | List> pointsFromChunks = chunksContainingData.parallelStream() 175 | .map(chunk -> chunk.query(query)) 176 | .collect(toList()); 177 | 178 | // Group results by metric name 179 | Map>> pointsByMetricName = pointsFromChunks.stream() 180 | .flatMap(List::stream) 181 | .map(series -> new AbstractMap.SimpleEntry<>(series.getMetric(), series.getPoints())) 182 | .collect(groupingBy(Map.Entry::getKey, mapping(Map.Entry::getValue, toList()))); 183 | 184 | if (queryAggregation.equals(QueryAggregation.ZIMSUM)) { 185 | // Run zimsum 186 | Map pointsZimsum = pointsByMetricName.values().stream() 187 | .flatMap(List::stream) 188 | .flatMap(List::stream) 189 | .collect(Collectors.groupingBy(Point::getTs, 190 | Collectors.summingDouble(Point::getVal))); 191 | 192 | List aggregatedPoints = pointsZimsum.entrySet().stream() 193 | .map(entry -> new Point(entry.getKey(), entry.getValue())) 194 | .collect(toList()); 195 | 196 | return Arrays.asList( 197 | new TimeSeries(queryAggregation + " " + query.toString(), aggregatedPoints)); 198 | } else { 199 | // Merge the points into a single timeseries 200 | return pointsByMetricName.entrySet().stream() 201 | .map(metricKey -> { 202 | List points = metricKey.getValue().stream() 203 | .flatMap(List::stream) 204 | .collect(toList()); 205 | return new TimeSeries(metricKey.getKey(), Points.dedup(points)); 206 | }) 207 | .collect(toList()); 208 | } 209 | } 210 | 211 | /* 212 | * Query multiple chunks that contain data between scartTs and endTs, merge their results. Return 213 | * a single list of time series events. 214 | */ 215 | public List query(Query query, long startTsSecs, long endTsSecs, 216 | QueryAggregation queryAggregation) { 217 | return queryAroundChunkBoundaries(query, startTsSecs, endTsSecs, queryAggregation); 218 | } 219 | 220 | @VisibleForTesting 221 | Map getChunkMap() { 222 | return chunkMap; 223 | } 224 | 225 | /** 226 | * This code is only called during tests and benchmarks. So, the big sync lock is not an issue 227 | * in practice. 228 | */ 229 | @VisibleForTesting 230 | void toOffHeapChunkMap() { 231 | Map offHeapChunkMap = new ConcurrentHashMap<>(); 232 | synchronized (chunkMapSync) { 233 | chunkMap.entrySet().stream().forEach(chunkEntry -> { 234 | LOG.info("Moving chunk to off heap: {}", chunkEntry.getValue().info()); 235 | Chunk offHeapChunk = toOffHeapChunk(chunkEntry.getValue()); 236 | offHeapChunkMap.put(chunkEntry.getKey(), offHeapChunk); 237 | LOG.info("Moved chunk to off heap: {}", chunkEntry.getValue().info()); 238 | }); 239 | 240 | chunkMap.clear(); 241 | chunkMap.putAll(offHeapChunkMap); 242 | LOG.info("Moved all chunks to off heap."); 243 | } 244 | } 245 | 246 | private Chunk toOffHeapChunk(Chunk chunk) { 247 | ChunkImpl chunkImpl = (ChunkImpl) chunk; 248 | MetricsAndTagStoreImpl metricsAndTagStore = (MetricsAndTagStoreImpl) chunkImpl.getStore(); 249 | Map seriesMap = (metricsAndTagStore).getMetricStore().getSeriesMap(); 250 | 251 | MetricStore offHeapMetricStore = 252 | OffHeapVarBitMetricStore.toOffHeapStore(seriesMap, chunk.info().dataSet, dataDirectory); 253 | 254 | MetricAndTagStore newMetricAndTagStore = 255 | new MetricsAndTagStoreImpl(metricsAndTagStore.getTagStore(), offHeapMetricStore); 256 | 257 | return new ChunkImpl(newMetricAndTagStore, chunkImpl.info()); 258 | } 259 | 260 | public void toReadOnlyChunks(List> expiredChunks) { 261 | LOG.info("Chunks past on heap cut off are: {}", expiredChunks); 262 | 263 | expiredChunks.forEach(entry -> { 264 | try { 265 | if (chunkMap.containsKey(entry.getKey())) { 266 | final Chunk chunk = entry.getValue(); 267 | LOG.info("Moving chunk {} to off heap.", chunk.info()); 268 | 269 | // Set the chunk to read only before moving it off heap so it can't be changed. 270 | chunk.setReadOnly(true); 271 | Chunk readOnlyChunk = toOffHeapChunk(chunk); 272 | 273 | synchronized (chunkMapSync) { 274 | Chunk oldChunk = chunkMap.put(entry.getKey(), readOnlyChunk); 275 | // Close the old chunk to free up memory faster. 276 | oldChunk.close(); 277 | } 278 | 279 | LOG.info("Moved chunk {} to off heap.", chunk.info()); 280 | } else { 281 | LOG.warn("Possible bug or race condition! Chunk {} doesn't exist in chunk map {}.", 282 | entry, chunkMap); 283 | } 284 | } catch (Exception e) { 285 | LOG.error("Exception when moving a chunk {} off heap.", entry.getKey(), e); 286 | } 287 | }); 288 | } 289 | 290 | public void removeStaleChunks(List> staleChunks) { 291 | LOG.info("Stale chunks to be removed are: {}", staleChunks); 292 | 293 | if (chunkMap.isEmpty()) { 294 | LOG.warn("Possible bug or race condition. There are no chunks in chunk map."); 295 | } 296 | 297 | staleChunks.forEach(entry -> { 298 | try { 299 | if (chunkMap.containsKey(entry.getKey())) { 300 | final Chunk chunk = entry.getValue(); 301 | String chunkInfo = chunk.info().toString(); 302 | LOG.info("Deleting chunk {}.", chunkInfo); 303 | 304 | synchronized (chunkMapSync) { 305 | chunkMap.remove(entry.getKey()); 306 | } 307 | // Close the chunk to free up resources. 308 | chunk.close(); 309 | LOG.info("Deleted chunk {}.", chunkInfo); 310 | } else { 311 | LOG.warn("Possible bug or race condition! Chunk {} doesn't exist in chunk map {}.", 312 | entry, chunkMap); 313 | } 314 | } catch (Exception e) { 315 | LOG.error("Exception when deleting chunk.", e); 316 | } 317 | }); 318 | } 319 | } 320 | --------------------------------------------------------------------------------