/** * Gets the ID for the provided {@link AppliedPTransform}. The provided {@link AppliedPTransform} * will not be added to the components produced by this {@link SdkComponents} until it is * translated via {@link #registerPTransform(AppliedPTransform, List)}. */ private String getApplicationName(AppliedPTransform<?, ?, ?> appliedPTransform) { String existing = transformIds.get(appliedPTransform); if (existing != null) { return existing; } String name = appliedPTransform.getFullName(); if (name.isEmpty()) { name = "unnamed-ptransform"; } name = uniqify(name, transformIds.values()); transformIds.put(appliedPTransform, name); return name; }
/** * Registers the provided {@link Environment} into this {@link SdkComponents}, returning a unique * ID for the {@link Environment}. Multiple registrations of the same {@link Environment} will * return the same unique ID. */ public String registerEnvironment(Environment env) { String existing = environmentIds.get(env); if (existing != null) { return existing; } String name = uniqify(env.getUrn(), environmentIds.values()); environmentIds.put(env, name); componentsBuilder.putEnvironments(name, env); return name; }
/** * Registers the provided {@link PCollection} into this {@link SdkComponents}, returning a unique * ID for the {@link PCollection}. Multiple registrations of the same {@link PCollection} will * return the same unique ID. */ public String registerPCollection(PCollection<?> pCollection) throws IOException { String existing = pCollectionIds.get(pCollection); if (existing != null) { return existing; } String uniqueName = uniqify(pCollection.getName(), pCollectionIds.values()); pCollectionIds.put(pCollection, uniqueName); componentsBuilder.putPcollections( uniqueName, PCollectionTranslation.toProto(pCollection, this)); return uniqueName; }
/** * Registers the provided {@link WindowingStrategy} into this {@link SdkComponents}, returning a * unique ID for the {@link WindowingStrategy}. Multiple registrations of the same {@link * WindowingStrategy} will return the same unique ID. */ public String registerWindowingStrategy(WindowingStrategy<?, ?> windowingStrategy) throws IOException { String existing = windowingStrategyIds.get(windowingStrategy); if (existing != null) { return existing; } String baseName = String.format( "%s(%s)", NameUtils.approximateSimpleName(windowingStrategy), NameUtils.approximateSimpleName(windowingStrategy.getWindowFn())); String name = uniqify(baseName, windowingStrategyIds.values()); windowingStrategyIds.put(windowingStrategy, name); RunnerApi.WindowingStrategy windowingStrategyProto = WindowingStrategyTranslation.toProto(windowingStrategy, this); componentsBuilder.putWindowingStrategies(name, windowingStrategyProto); return name; }
/** * Registers the provided {@link Coder} into this {@link SdkComponents}, returning a unique ID for * the {@link Coder}. Multiple registrations of the same {@link Coder} will return the same unique * ID. * * <p>Coders are stored by identity to ensure that coders with implementations of {@link * #equals(Object)} and {@link #hashCode()} but incompatible binary formats are not considered the * same coder. */ public String registerCoder(Coder<?> coder) throws IOException { String existing = coderIds.get(Equivalence.identity().wrap(coder)); if (existing != null) { return existing; } String baseName = NameUtils.approximateSimpleName(coder); String name = uniqify(baseName, coderIds.values()); coderIds.put(Equivalence.identity().wrap(coder), name); RunnerApi.Coder coderProto = CoderTranslation.toProto(coder, this); componentsBuilder.putCoders(name, coderProto); return name; }