Collection<OperatorStateHandle> managedOperatorState = handle.getManagedOperatorState(); Collection<OperatorStateHandle> rawOperatorState = handle.getRawOperatorState(); Collection<KeyedStateHandle> managedKeyedState = handle.getManagedKeyedState(); Collection<KeyedStateHandle> rawKeyedState = handle.getRawKeyedState(); return new OperatorSubtaskState( new StateObjectCollection<>(mergedManagedOperatorState), new StateObjectCollection<>(mergedRawOperatorState),
Assert.assertTrue(mode > ONLY_JM_RECOVERY == (taskLocalState != null && taskLocalState.hasState())); jobManagerOwnedState.getManagedKeyedState().discardState(); } else if (mode == JM_REMOVE_TM_RECOVERY) { taskLocalState.getManagedKeyedState().discardState();
Collection<OperatorStateHandle> rawOperatorState = state.getRawOperatorState(); if (rawOperatorState != null) { dos.writeInt(rawOperatorState.size()); Collection<OperatorStateHandle> managedOperatorState = state.getManagedOperatorState(); if (managedOperatorState != null) { dos.writeInt(managedOperatorState.size()); Collection<KeyedStateHandle> rawKeyedState = state.getRawKeyedState(); if (rawKeyedState != null) { dos.writeInt(rawKeyedState.size()); Collection<KeyedStateHandle> managedKeyedState = state.getManagedKeyedState(); if (managedKeyedState != null) { dos.writeInt(managedKeyedState.size());
OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( new OperatorStreamStateHandle( Collections.singletonMap( verify(operatorStateBackend).restore(eq(operatorSubtaskState.getManagedOperatorState())); verify(keyedStateBackend).restore(eq(operatorSubtaskState.getManagedKeyedState()));
public OperatorSnapshotFinalizer( @Nonnull OperatorSnapshotFutures snapshotFutures) throws ExecutionException, InterruptedException { SnapshotResult<KeyedStateHandle> keyedManaged = FutureUtils.runIfNotDoneAndGet(snapshotFutures.getKeyedStateManagedFuture()); SnapshotResult<KeyedStateHandle> keyedRaw = FutureUtils.runIfNotDoneAndGet(snapshotFutures.getKeyedStateRawFuture()); SnapshotResult<OperatorStateHandle> operatorManaged = FutureUtils.runIfNotDoneAndGet(snapshotFutures.getOperatorStateManagedFuture()); SnapshotResult<OperatorStateHandle> operatorRaw = FutureUtils.runIfNotDoneAndGet(snapshotFutures.getOperatorStateRawFuture()); jobManagerOwnedState = new OperatorSubtaskState( operatorManaged.getJobManagerOwnedSnapshot(), operatorRaw.getJobManagerOwnedSnapshot(), keyedManaged.getJobManagerOwnedSnapshot(), keyedRaw.getJobManagerOwnedSnapshot() ); taskLocalState = new OperatorSubtaskState( operatorManaged.getTaskLocalSnapshot(), operatorRaw.getTaskLocalSnapshot(), keyedManaged.getTaskLocalSnapshot(), keyedRaw.getTaskLocalSnapshot() ); }
private void collectPartionableStates( List<OperatorState> operatorStates, List<List<OperatorStateHandle>> managedOperatorStates, List<List<OperatorStateHandle>> rawOperatorStates) { for (OperatorState operatorState : operatorStates) { List<OperatorStateHandle> managedOperatorState = null; List<OperatorStateHandle> rawOperatorState = null; for (int i = 0; i < operatorState.getParallelism(); i++) { OperatorSubtaskState operatorSubtaskState = operatorState.getState(i); if (operatorSubtaskState != null) { if (operatorSubtaskState.getManagedOperatorState() != null) { if (managedOperatorState == null) { managedOperatorState = new ArrayList<>(); } managedOperatorState.add(operatorSubtaskState.getManagedOperatorState()); } if (operatorSubtaskState.getRawOperatorState() != null) { if (rawOperatorState == null) { rawOperatorState = new ArrayList<>(); } rawOperatorState.add(operatorSubtaskState.getRawOperatorState()); } } } managedOperatorStates.add(managedOperatorState); rawOperatorStates.add(rawOperatorState); } }
private Tuple2<Collection<KeyedStateHandle>, Collection<KeyedStateHandle>> reAssignSubKeyedStates( OperatorState operatorState, List<KeyGroupRange> keyGroupPartitions, int subTaskIndex, int newParallelism, int oldParallelism) { Collection<KeyedStateHandle> subManagedKeyedState; Collection<KeyedStateHandle> subRawKeyedState; if (newParallelism == oldParallelism) { if (operatorState.getState(subTaskIndex) != null) { subManagedKeyedState = operatorState.getState(subTaskIndex).getManagedKeyedState(); subRawKeyedState = operatorState.getState(subTaskIndex).getRawKeyedState(); } else { subManagedKeyedState = Collections.emptyList(); subRawKeyedState = Collections.emptyList(); } } else { subManagedKeyedState = getManagedKeyedStateHandles(operatorState, keyGroupPartitions.get(subTaskIndex)); subRawKeyedState = getRawKeyedStateHandles(operatorState, keyGroupPartitions.get(subTaskIndex)); } if (subManagedKeyedState.isEmpty() && subRawKeyedState.isEmpty()) { return new Tuple2<>(Collections.emptyList(), Collections.emptyList()); } else { return new Tuple2<>(subManagedKeyedState, subRawKeyedState); } }
public static Optional<KeyedBackendSerializationProxy<?>> getKeyedBackendSerializationProxy(OperatorState opState) { try { KeyedStateHandle firstHandle = opState.getStates().iterator().next().getManagedKeyedState().iterator() .next(); if (firstHandle instanceof IncrementalKeyedStateHandle) { return Optional.of(getKeyedBackendSerializationProxy( ((IncrementalKeyedStateHandle) firstHandle).getMetaStateHandle())); } else { return Optional.of(getKeyedBackendSerializationProxy((StreamStateHandle) firstHandle)); } } catch (Exception e) { return Optional.empty(); } }
/** * Returns true if at least one {@link OperatorSubtaskState} in subtaskStatesByOperatorID has state. */ public boolean hasState() { for (OperatorSubtaskState operatorSubtaskState : subtaskStatesByOperatorID.values()) { if (operatorSubtaskState != null && operatorSubtaskState.hasState()) { return true; } } return false; }
/** * Collect {@link KeyGroupsStateHandle rawKeyedStateHandles} which have intersection with given * {@link KeyGroupRange} from {@link TaskState operatorState} * * @param operatorState all state handles of a operator * @param subtaskKeyGroupRange the KeyGroupRange of a subtask * @return all rawKeyedStateHandles which have intersection with given KeyGroupRange */ public static List<KeyedStateHandle> getRawKeyedStateHandles( OperatorState operatorState, KeyGroupRange subtaskKeyGroupRange) { List<KeyedStateHandle> subtaskKeyedStateHandles = null; for (int i = 0; i < operatorState.getParallelism(); i++) { if (operatorState.getState(i) != null && operatorState.getState(i).getRawKeyedState() != null) { KeyedStateHandle intersectedKeyedStateHandle = operatorState.getState(i).getRawKeyedState().getIntersection(subtaskKeyGroupRange); if (intersectedKeyedStateHandle != null) { if (subtaskKeyedStateHandles == null) { subtaskKeyedStateHandles = new ArrayList<>(); } subtaskKeyedStateHandles.add(intersectedKeyedStateHandle); } } } return subtaskKeyedStateHandles; }
@Override public void registerSharedStates(SharedStateRegistry sharedStateRegistry) { for (OperatorSubtaskState operatorSubtaskState : operatorSubtaskStates.values()) { operatorSubtaskState.registerSharedStates(sharedStateRegistry); } }
@Override public long getStateSize() { long result = 0L; for (int i = 0; i < parallelism; i++) { OperatorSubtaskState operatorSubtaskState = operatorSubtaskStates.get(i); if (operatorSubtaskState != null) { result += operatorSubtaskState.getStateSize(); } } return result; }
/** * Restores the OperatorStateBackend corresponding to the given subtask. The * backend is completely restored in-memory. */ public OperatorStateBackend createOperatorStateBackendFromSnapshot(int subtask) throws Exception { return restoreOperatorStateBackend(opState.getState(subtask).getManagedOperatorState()); }
@Override public void discardState() throws Exception { for (OperatorSubtaskState operatorSubtaskState : operatorSubtaskStates.values()) { operatorSubtaskState.discardState(); } }
return new OperatorSubtaskState( new StateObjectCollection<>(managedOperatorState), new StateObjectCollection<>(rawOperatorState),
private void collectPartionableStates( List<OperatorState> operatorStates, List<List<OperatorStateHandle>> managedOperatorStates, List<List<OperatorStateHandle>> rawOperatorStates) { for (OperatorState operatorState : operatorStates) { List<OperatorStateHandle> managedOperatorState = null; List<OperatorStateHandle> rawOperatorState = null; for (int i = 0; i < operatorState.getParallelism(); i++) { OperatorSubtaskState operatorSubtaskState = operatorState.getState(i); if (operatorSubtaskState != null) { if (managedOperatorState == null) { managedOperatorState = new ArrayList<>(); } managedOperatorState.addAll(operatorSubtaskState.getManagedOperatorState()); if (rawOperatorState == null) { rawOperatorState = new ArrayList<>(); } rawOperatorState.addAll(operatorSubtaskState.getRawOperatorState()); } } managedOperatorStates.add(managedOperatorState); rawOperatorStates.add(rawOperatorState); } }
private Tuple2<Collection<KeyedStateHandle>, Collection<KeyedStateHandle>> reAssignSubKeyedStates( OperatorState operatorState, List<KeyGroupRange> keyGroupPartitions, int subTaskIndex, int newParallelism, int oldParallelism) { Collection<KeyedStateHandle> subManagedKeyedState; Collection<KeyedStateHandle> subRawKeyedState; if (newParallelism == oldParallelism) { if (operatorState.getState(subTaskIndex) != null) { KeyedStateHandle oldSubManagedKeyedState = operatorState.getState(subTaskIndex).getManagedKeyedState(); KeyedStateHandle oldSubRawKeyedState = operatorState.getState(subTaskIndex).getRawKeyedState(); subManagedKeyedState = oldSubManagedKeyedState != null ? Collections.singletonList( oldSubManagedKeyedState) : null; subRawKeyedState = oldSubRawKeyedState != null ? Collections.singletonList( oldSubRawKeyedState) : null; } else { subManagedKeyedState = null; subRawKeyedState = null; } } else { subManagedKeyedState = getManagedKeyedStateHandles(operatorState, keyGroupPartitions.get(subTaskIndex)); subRawKeyedState = getRawKeyedStateHandles(operatorState, keyGroupPartitions.get(subTaskIndex)); } if (subManagedKeyedState == null && subRawKeyedState == null) { return null; } return new Tuple2<>(subManagedKeyedState, subRawKeyedState); }
.getManagedKeyedState() .stream() .map(keyedStateHandle -> {
/** * Returns true if at least one {@link OperatorSubtaskState} in subtaskStatesByOperatorID has state. */ public boolean hasState() { for (OperatorSubtaskState operatorSubtaskState : subtaskStatesByOperatorID.values()) { if (operatorSubtaskState != null && operatorSubtaskState.hasState()) { return true; } } return false; }
/** * Collect {@link KeyGroupsStateHandle rawKeyedStateHandles} which have intersection with given * {@link KeyGroupRange} from {@link TaskState operatorState} * * @param operatorState all state handles of a operator * @param subtaskKeyGroupRange the KeyGroupRange of a subtask * @return all rawKeyedStateHandles which have intersection with given KeyGroupRange */ public static List<KeyedStateHandle> getRawKeyedStateHandles( OperatorState operatorState, KeyGroupRange subtaskKeyGroupRange) { final int parallelism = operatorState.getParallelism(); List<KeyedStateHandle> extractedKeyedStateHandles = null; for (int i = 0; i < parallelism; i++) { if (operatorState.getState(i) != null) { Collection<KeyedStateHandle> rawKeyedState = operatorState.getState(i).getRawKeyedState(); if (extractedKeyedStateHandles == null) { extractedKeyedStateHandles = new ArrayList<>(parallelism * rawKeyedState.size()); } extractIntersectingState( rawKeyedState, subtaskKeyGroupRange, extractedKeyedStateHandles); } } return extractedKeyedStateHandles; }