private <T> ValueInSingleWindow<T> wrapValue(T value) { return ValueInSingleWindow.of( value, GlobalWindow.TIMESTAMP_MAX_VALUE, GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING); }
@Override public void encode(ValueInSingleWindow<T> windowedElem, OutputStream outStream, Context context) throws IOException { InstantCoder.of().encode(windowedElem.getTimestamp(), outStream); windowCoder.encode(windowedElem.getWindow(), outStream); PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream); valueCoder.encode(windowedElem.getValue(), outStream, context); }
/** @deprecated Use {@link TestPipeline} with the {@code DirectRunner}. */ @Deprecated public List<TimestampedValue<OutputT>> peekOutputElementsWithTimestamp() { // TODO: Should we return an unmodifiable list? return getImmutableOutput(mainOutputTag) .stream() .map(input -> TimestampedValue.of(input.getValue(), input.getTimestamp())) .collect(Collectors.toList()); }
/** @deprecated Use {@link TestPipeline} with the {@code DirectRunner}. */ @Deprecated public List<TimestampedValue<OutputT>> peekOutputElementsInWindow( TupleTag<OutputT> tag, BoundedWindow window) { ImmutableList.Builder<TimestampedValue<OutputT>> valuesBuilder = ImmutableList.builder(); for (ValueInSingleWindow<OutputT> value : getImmutableOutput(tag)) { if (value.getWindow().equals(window)) { valuesBuilder.add(TimestampedValue.of(value.getValue(), value.getTimestamp())); } } return valuesBuilder.build(); }
@Override public Long getDestination(ValueInSingleWindow<WeatherData> elem) { return elem.getValue().year; }
@Override public Integer getDestination(ValueInSingleWindow<String> element) { assertThat( element.getWindow(), Matchers.instanceOf(PartitionedGlobalWindow.class)); Matcher matcher = userPattern.matcher(element.getValue()); if (matcher.matches()) { // Since we name tables by userid, we can simply store an Integer to represent // a table. return Integer.valueOf(matcher.group(2)); } throw new RuntimeException("Unmatching destination " + element.getValue()); }
/** Writes the accumulated rows into BigQuery with streaming API. */ @FinishBundle public void finishBundle(FinishBundleContext context) throws Exception { List<ValueInSingleWindow<ErrorT>> failedInserts = Lists.newArrayList(); BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); for (Map.Entry<String, List<ValueInSingleWindow<TableRow>>> entry : tableRows.entrySet()) { TableReference tableReference = BigQueryHelpers.parseTableSpec(entry.getKey()); flushRows( tableReference, entry.getValue(), uniqueIdsForTableRows.get(entry.getKey()), options, failedInserts); } tableRows.clear(); uniqueIdsForTableRows.clear(); for (ValueInSingleWindow<ErrorT> row : failedInserts) { context.output(failedOutputTag, row.getValue(), row.getTimestamp(), row.getWindow()); } }
@Override public Iterable<T> apply(Iterable<ValueInSingleWindow<T>> input) { List<T> outputs = new ArrayList<>(); for (ValueInSingleWindow<T> value : input) { outputs.add(value.getValue()); } return outputs; } }
@ProcessElement public void processElement( @Element T element, @Timestamp Instant timestamp, BoundedWindow window, PaneInfo pane, OutputReceiver<ValueInSingleWindow<T>> r) { r.outputWithTimestamp( ValueInSingleWindow.of(element, timestamp, window, pane), timestamp); } }))
public String getDestination(ValueInSingleWindow<FeatureRowExtended> element) { FeatureRowExtended featureRowExtended = element.getValue(); FeatureRow row = featureRowExtended.getRow(); EntitySpec entityInfo = specs.getEntitySpec(row.getEntityName()); Granularity.Enum granularity = row.getGranularity(); String tableName = entityInfo.getName() + "_" + granularity.name().toLowerCase(); return bigQueryOptions.project + ":" + bigQueryOptions.dataset + "." + tableName; }
public long insertAll(TableReference ref, List<TableRow> rowList) throws IOException, InterruptedException { List<ValueInSingleWindow<TableRow>> rows = rowList.stream() .map(r -> ValueInSingleWindow.of( r, BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)) .collect(Collectors.toList()); return bqServices.getDatasetService(bqOptions) .insertAll(ref, rows, null, InsertRetryPolicy.alwaysRetry(), new ArrayList<>(), ErrorContainer.TABLE_ROW_ERROR_CONTAINER, false, false); }
TableRow row = rowsToPublish.get(i).getValue(); TableDataInsertAllRequest.Rows out = new TableDataInsertAllRequest.Rows(); if (idsToPublish != null) {
public long insertAll( TableReference ref, List<TableRow> rowList, @Nullable List<String> insertIdList) throws IOException, InterruptedException { List<ValueInSingleWindow<TableRow>> windowedRows = Lists.newArrayList(); for (TableRow row : rowList) { windowedRows.add( ValueInSingleWindow.of( row, GlobalWindow.TIMESTAMP_MAX_VALUE, GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } return insertAll( ref, windowedRows, insertIdList, InsertRetryPolicy.alwaysRetry(), null, null, false, false); }
@Override public Iterable<T> apply(Iterable<ValueInSingleWindow<T>> input) { List<T> outputs = new ArrayList<>(); for (ValueInSingleWindow<T> value : input) { if (!value.getPane().isFirst() || !value.getPane().isLast()) { throw site.wrap( String.format( "Expected elements to be produced by a trigger that fires at most once, but got " + "a value %s in a pane that is %s.", value, value.getPane().isFirst() ? "not the last pane" : "not the first pane")); } outputs.add(value.getValue()); } return outputs; } }
BigQueryHelpers.stripPartitionDecorator(ref.getTableId())); for (int i = 0; i < rowList.size(); ++i) { TableRow row = rowList.get(i).getValue(); List<TableDataInsertAllResponse.InsertErrors> allErrors = insertErrors.get(row); boolean shouldInsert = true;