@Override public String apply(long value) { final long truncated = granularity.bucketStart(DateTimes.utc(value)).getMillis(); return formatter == null ? String.valueOf(truncated) : formatter.print(truncated); }
public DateTime bucketEnd(DateTime time) { return increment(bucketStart(time)); }
@Override public int compare(Result<T> r1, Result<T> r2) { return Longs.compare( gran.bucketStart(r1.getTimestamp()).getMillis(), gran.bucketStart(r2.getTimestamp()).getMillis() ); }
@Override public long apply(Row row) { return query.getGranularity().bucketStart(row.getTimestamp()).getMillis(); } };
private IntervalIterator(Interval inputInterval) { this.inputInterval = inputInterval; currStart = bucketStart(inputInterval.getStart()); currEnd = increment(currStart); }
/** * Return a granularity-sized Interval containing a particular DateTime. */ public final Interval bucket(DateTime t) { DateTime start = bucketStart(t); return new Interval(start, increment(start)); }
@Override protected void innerMap( InputRow inputRow, Context context ) throws IOException, InterruptedException { final List<Object> groupKey = Rows.toGroupKey( rollupGranularity.bucketStart(inputRow.getTimestamp()).getMillis(), inputRow ); context.write( new BytesWritable(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(groupKey)), NullWritable.get() ); context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_COUNTER).increment(1); } }
@Nullable private Comparator<Row> getTimeComparator(boolean granular) { if (Granularities.ALL.equals(getGranularity())) { return null; } else if (granular) { return (lhs, rhs) -> Longs.compare( getGranularity().bucketStart(lhs.getTimestamp()).getMillis(), getGranularity().bucketStart(rhs.getTimestamp()).getMillis() ); } else { return NON_GRANULAR_TIME_COMP; } }
private DateTime getAllowedMinTime() { final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); final Period windowPeriod = config.getWindowPeriod(); final long windowMillis = windowPeriod.toStandardDuration().getMillis(); return segmentGranularity.bucketStart( DateTimes.utc(Math.max(windowMillis, rejectionPolicy.getCurrMaxTime().getMillis()) - windowMillis) ); }
@Override public int compare(DataSegment lhs, DataSegment rhs) { int retVal; DateTime lhsMonth = Granularities.MONTH.bucketStart(lhs.getInterval().getStart()); DateTime rhsMonth = Granularities.MONTH.bucketStart(rhs.getInterval().getStart()); retVal = lhsMonth.compareTo(rhsMonth); if (retVal != 0) { return retVal; } return lhs.compareTo(rhs); } };
private DateTime adjustTimestamp(final Row row) { if (query.getGranularity() instanceof AllGranularity) { return row.getTimestamp(); } else { return query.getGranularity().bucketStart(row.getTimestamp()); } } }
/** * Get the proper bucket for some input row. * * @param inputRow an InputRow * * @return the Bucket that this row belongs to */ public Optional<Bucket> getBucket(InputRow inputRow) { final Optional<Interval> timeBucket = schema.getDataSchema().getGranularitySpec().bucketInterval( DateTimes.utc(inputRow.getTimestampFromEpoch()) ); if (!timeBucket.isPresent()) { return Optional.absent(); } final DateTime bucketStart = timeBucket.get().getStart(); final ShardSpec actualSpec = shardSpecLookups.get(bucketStart.getMillis()) .getShardSpec( rollupGran.bucketStart(inputRow.getTimestamp()).getMillis(), inputRow ); final HadoopyShardSpec hadoopyShardSpec = hadoopShardSpecLookup.get(bucketStart.getMillis()).get(actualSpec); return Optional.of( new Bucket( hadoopyShardSpec.getShardNum(), bucketStart, actualSpec.getPartitionNum() ) ); }
@Test public void testTruncate() { DateTime date = DateTimes.of("2011-03-15T22:42:23.898"); Assert.assertEquals(DateTimes.of("2011-01-01T00:00:00.000"), YEAR.bucketStart(date)); Assert.assertEquals(DateTimes.of("2011-03-01T00:00:00.000"), MONTH.bucketStart(date)); Assert.assertEquals(DateTimes.of("2011-03-14T00:00:00.000"), WEEK.bucketStart(date)); Assert.assertEquals(DateTimes.of("2011-03-15T00:00:00.000"), DAY.bucketStart(date)); Assert.assertEquals(DateTimes.of("2011-03-15T22:00:00.000"), HOUR.bucketStart(date)); Assert.assertEquals(DateTimes.of("2011-03-15T22:42:00.000"), MINUTE.bucketStart(date)); Assert.assertEquals(DateTimes.of("2011-03-15T22:42:23.000"), SECOND.bucketStart(date)); }
rollupGranularity.bucketStart(inputRow.getTimestamp()).getMillis(), inputRow );
@Override public SegmentIdWithShardSpec allocate( final InputRow row, final String sequenceName, final String previousSegmentId, final boolean skipSegmentLineageCheck ) { synchronized (counters) { DateTime dateTimeTruncated = granularity.bucketStart(row.getTimestamp()); final long timestampTruncated = dateTimeTruncated.getMillis(); if (!counters.containsKey(timestampTruncated)) { counters.put(timestampTruncated, new AtomicInteger()); } final int partitionNum = counters.get(timestampTruncated).getAndIncrement(); return new SegmentIdWithShardSpec( dataSource, granularity.bucket(dateTimeTruncated), VERSION, new NumberedShardSpec(partitionNum, 0) ); } } }
@Test public void testDurationTruncate() { { final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00"); Granularity gran = new DurationGranularity( new Period("PT12H5M").toStandardDuration().getMillis(), origin ); assertSameDateTime( Lists.newArrayList( DateTimes.of("2012-01-01T04:50:00.000-08:00"), DateTimes.of("2012-01-02T05:00:00.000-08:00"), DateTimes.of("2012-01-02T17:05:00.000-08:00"), DateTimes.of("2012-02-03T22:25:00.000-08:00") ), Lists.newArrayList( gran.bucketStart(DateTimes.of("2012-01-01T05:00:04.123-08:00")), gran.bucketStart(DateTimes.of("2012-01-02T07:00:04.123-08:00")), gran.bucketStart(DateTimes.of("2012-01-03T00:20:04.123-08:00")), gran.bucketStart(DateTimes.of("2012-02-03T22:25:00.000-08:00")) ) ); } }
private SegmentIdWithShardSpec getSegmentIdentifier(long timestamp) { if (!rejectionPolicy.accept(timestamp)) { return null; } final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); final VersioningPolicy versioningPolicy = config.getVersioningPolicy(); DateTime truncatedDateTime = segmentGranularity.bucketStart(DateTimes.utc(timestamp)); final long truncatedTime = truncatedDateTime.getMillis(); SegmentIdWithShardSpec retVal = segments.get(truncatedTime); if (retVal == null) { final Interval interval = new Interval( truncatedDateTime, segmentGranularity.increment(truncatedDateTime) ); retVal = new SegmentIdWithShardSpec( schema.getDataSource(), interval, versioningPolicy.getVersion(interval), config.getShardSpec() ); addSegment(retVal); } return retVal; }
@Override public Result<TimeseriesResultValue> apply(Result<TimeseriesResultValue> arg1, Result<TimeseriesResultValue> arg2) { if (arg1 == null) { return arg2; } if (arg2 == null) { return arg1; } TimeseriesResultValue arg1Val = arg1.getValue(); TimeseriesResultValue arg2Val = arg2.getValue(); Map<String, Object> retVal = new LinkedHashMap<String, Object>(); for (AggregatorFactory factory : aggregations) { final String metricName = factory.getName(); retVal.put(metricName, factory.combine(arg1Val.getMetric(metricName), arg2Val.getMetric(metricName))); } return (gran instanceof AllGranularity) ? new Result<TimeseriesResultValue>( arg1.getTimestamp(), new TimeseriesResultValue(retVal) ) : new Result<TimeseriesResultValue>( gran.bucketStart(arg1.getTimestamp()), new TimeseriesResultValue(retVal) ); }
private Sink getSink(long timestamp) { if (!rejectionPolicy.accept(timestamp)) { return null; } final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); final VersioningPolicy versioningPolicy = config.getVersioningPolicy(); DateTime truncatedDateTime = segmentGranularity.bucketStart(DateTimes.utc(timestamp)); final long truncatedTime = truncatedDateTime.getMillis(); Sink retVal = sinks.get(truncatedTime); if (retVal == null) { final Interval sinkInterval = new Interval( truncatedDateTime, segmentGranularity.increment(truncatedDateTime) ); retVal = new Sink( sinkInterval, schema, config.getShardSpec(), versioningPolicy.getVersion(sinkInterval), config.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()), config.isReportParseExceptions(), config.getDedupColumn() ); addSink(retVal); } return retVal; }
: gran.bucketStart(arg1.getTimestamp());