public DateTime bucketEnd(DateTime time) { return increment(bucketStart(time)); }
rollupGranularity.bucketStart(inputRow.getTimestamp()).getMillis(), inputRow ); interval = config.getGranularitySpec() .getSegmentGranularity() .bucket(DateTimes.utc(inputRow.getTimestampFromEpoch()));
@JsonCreator public UniformGranularitySpec( @JsonProperty("segmentGranularity") Granularity segmentGranularity, @JsonProperty("queryGranularity") Granularity queryGranularity, @JsonProperty("rollup") Boolean rollup, @JsonProperty("intervals") List<Interval> inputIntervals ) { this.queryGranularity = queryGranularity == null ? DEFAULT_QUERY_GRANULARITY : queryGranularity; this.rollup = rollup == null ? Boolean.TRUE : rollup; this.segmentGranularity = segmentGranularity == null ? DEFAULT_SEGMENT_GRANULARITY : segmentGranularity; if (inputIntervals != null) { List<Interval> granularIntervals = new ArrayList<>(); for (Interval inputInterval : inputIntervals) { Iterables.addAll(granularIntervals, this.segmentGranularity.getIterable(inputInterval)); } this.inputIntervals = ImmutableList.copyOf(inputIntervals); this.wrappedSpec = new ArbitraryGranularitySpec(queryGranularity, rollup, granularIntervals); } else { this.inputIntervals = null; this.wrappedSpec = null; } }
private SegmentIdWithShardSpec tryAllocateFirstSegment(TaskActionToolbox toolbox, Task task, Interval rowInterval) { // No existing segments for this row, but there might still be nearby ones that conflict with our preferred // segment granularity. Try that first, and then progressively smaller ones if it fails. final List<Interval> tryIntervals = Granularity.granularitiesFinerThan(preferredSegmentGranularity) .stream() .map(granularity -> granularity.bucket(timestamp)) .collect(Collectors.toList()); for (Interval tryInterval : tryIntervals) { if (tryInterval.contains(rowInterval)) { final SegmentIdWithShardSpec identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false); if (identifier != null) { return identifier; } } } return null; }
for (Interval interval : dataGranularity.getIterable(inputInterval)) { intervals.add(trim(inputInterval, interval)); intervalPath = customFormatter.print(t); } else { intervalPath = dataGranularity.toPath(t);
@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 static List<Granularity> granularitiesFinerThan(final Granularity gran0) { final List<Granularity> retVal = new ArrayList<>(); final DateTime origin = (gran0 instanceof PeriodGranularity) ? ((PeriodGranularity) gran0).getOrigin() : null; final DateTimeZone tz = (gran0 instanceof PeriodGranularity) ? ((PeriodGranularity) gran0).getTimeZone() : null; for (GranularityType gran : GranularityType.values()) { /** * All and None are excluded b/c when asked to give all granularities finer * than "TEN_MINUTE", you want the answer to be "FIVE_MINUTE, MINUTE and SECOND" * it doesn't make sense to include ALL or None to be part of this. */ if (gran == GranularityType.ALL || gran == GranularityType.NONE) { continue; } final Granularity segmentGranularity = gran.create(origin, tz); final long segmentGranularityDurationMillis = segmentGranularity.bucket(DateTimes.EPOCH).toDurationMillis(); final long gran0DurationMillis = gran0.bucket(DateTimes.EPOCH).toDurationMillis(); if (segmentGranularityDurationMillis <= gran0DurationMillis) { retVal.add(segmentGranularity); } } retVal.sort((g1, g2) -> { long duration1 = g2.bucket(DateTimes.EPOCH).toDurationMillis(); long duration2 = g1.bucket(DateTimes.EPOCH).toDurationMillis(); return Longs.compare(duration1, duration2); }); return retVal; }
final DateTime key = segmentGranularity.toDate(status.getPath().toString()); final Long currVal = inputModifiedTimes.get(key.getMillis()); final long mTime = status.getModificationTime(); "%s/%s", config.getSchema().getIOConfig().getSegmentOutputPath(), segmentGranularity.toPath(timeBucket) ); for (FileStatus fileStatus : FSSpideringIterator.spiderIterable(fs, new Path(bucketOutput))) { if (fileStatus.getModificationTime() > mTime) { bucketsToRun.add(new Interval(timeBucket, segmentGranularity.increment(timeBucket))); break;
IncrementalIndexCursor( VirtualColumns virtualColumns, boolean descending, Filter filter, Interval interval, Interval actualInterval, Granularity gran ) { currEntry = new IncrementalIndexRowHolder(); columnSelectorFactory = new IncrementalIndexColumnSelectorFactory(index, virtualColumns, descending, currEntry); // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/incubator-druid/pull/6340 maxRowIndex = index.getLastRowIndex(); filterMatcher = filter == null ? BooleanValueMatcher.of(true) : filter.makeMatcher(columnSelectorFactory); numAdvanced = -1; final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis()); cursorIterable = index.getFacts().timeRangeIterable( descending, timeStart, Math.min(actualInterval.getEndMillis(), gran.increment(interval.getStart()).getMillis()) ); emptyRange = !cursorIterable.iterator().hasNext(); time = gran.toDateTime(interval.getStartMillis()); reset(); }
@Override public Interval next() { if (!hasNext()) { throw new NoSuchElementException("There are no more intervals"); } Interval retVal = new Interval(currStart, currEnd); currStart = currEnd; currEnd = increment(currStart); return retVal; }
@Override public Sequence<Cursor> makeCursors( @Nullable final Filter filter, final Interval interval, final VirtualColumns virtualColumns, final Granularity gran, final boolean descending, @Nullable QueryMetrics<?> queryMetrics ) { if (index.isEmpty()) { return Sequences.empty(); } final Interval dataInterval = new Interval(getMinTime(), gran.bucketEnd(getMaxTime())); if (!interval.overlaps(dataInterval)) { return Sequences.empty(); } final Interval actualInterval = interval.overlap(dataInterval); Iterable<Interval> intervals = gran.getIterable(actualInterval); if (descending) { intervals = Lists.reverse(ImmutableList.copyOf(intervals)); } return Sequences .simple(intervals) .map(i -> new IncrementalIndexCursor(virtualColumns, descending, filter, i, actualInterval, gran)); }
), pd.date, granularity.toDate(pd.path, formatter) ); pd.path ), granularity.toDate(pd.path), granularity.toDate(pd.path, formatter) ); ), pd.date, granularity.toDate(granularity.getFormatter(formatter).print(pd.date) + "/", formatter) ); boolean flag = false; try { granularity.toDate(pd.path, formatter);
public DateTime toDate(String filePath) { return toDate(filePath, Formatter.DEFAULT); }
public final String toPath(DateTime time) { return getFormatter(Formatter.DEFAULT).print(time); }
@Override public int compare(Result<T> r1, Result<T> r2) { return Longs.compare( gran.bucketStart(r1.getTimestamp()).getMillis(), gran.bucketStart(r2.getTimestamp()).getMillis() ); }
/** * Build a filter for an expression like FLOOR(column TO granularity) [operator] rhsMillis */ private static DimFilter buildTimeFloorFilter( final String column, final Granularity granularity, final SqlKind operatorKind, final long rhsMillis ) { final BoundRefKey boundRefKey = new BoundRefKey(column, null, StringComparators.NUMERIC); final Interval rhsInterval = granularity.bucket(DateTimes.utc(rhsMillis)); // Is rhs aligned on granularity boundaries? final boolean rhsAligned = rhsInterval.getStartMillis() == rhsMillis; return getBoundTimeDimFilter(operatorKind, boundRefKey, rhsInterval, rhsAligned); }
private SegmentIdentifier tryAllocateFirstSegment(TaskActionToolbox toolbox, Task task, Interval rowInterval) { // No existing segments for this row, but there might still be nearby ones that conflict with our preferred // segment granularity. Try that first, and then progressively smaller ones if it fails. final List<Interval> tryIntervals = Granularity.granularitiesFinerThan(preferredSegmentGranularity) .stream() .map(granularity -> granularity.bucket(timestamp)) .collect(Collectors.toList()); for (Interval tryInterval : tryIntervals) { if (tryInterval.contains(rowInterval)) { final SegmentIdentifier identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false); if (identifier != null) { return identifier; } } } return null; }
final DateTime key = segmentGranularity.toDate(status.getPath().toString()); final Long currVal = inputModifiedTimes.get(key.getMillis()); final long mTime = status.getModificationTime(); "%s/%s", config.getSchema().getIOConfig().getSegmentOutputPath(), segmentGranularity.toPath(timeBucket) ); for (FileStatus fileStatus : FSSpideringIterator.spiderIterable(fs, new Path(bucketOutput))) { if (fileStatus.getModificationTime() > mTime) { bucketsToRun.add(new Interval(timeBucket, segmentGranularity.increment(timeBucket))); break;
final long timeEnd = Math.min( interval.getEndMillis(), gran.increment(inputInterval.getStart()).getMillis() ); columnCache ); final DateTime myBucket = gran.toDateTime(inputInterval.getStartMillis());
public Interval getInterval() { DateTime min = DateTimes.utc(minTimestamp); return new Interval(min, isEmpty() ? min : gran.increment(DateTimes.utc(getMaxTimeMillis()))); }