Skip to content

Commit

Permalink
[FLINK-25511][state] Introduce StreamStateHandle ID
Browse files Browse the repository at this point in the history
The ID will be used to track state usage on TM;
State objects not shared with JM will be discarded.
  • Loading branch information
rkhachatryan committed Apr 28, 2022
1 parent c9b5e2f commit 5b58ef6
Show file tree
Hide file tree
Showing 17 changed files with 130 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,11 @@ public StateHandleID getStateHandleId() {
return stateHandleId;
}

@Override
public PhysicalStateHandleID getStreamStateHandleID() {
return stateHandle.getStreamStateHandleID();
}

@Override
public KeyGroupRange getKeyGroupRange() {
return groupRangeOffsets.getKeyGroupRange();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,11 @@ public Optional<byte[]> asBytesIfInMemory() {
return delegateStateHandle.asBytesIfInMemory();
}

@Override
public PhysicalStateHandleID getStreamStateHandleID() {
return delegateStateHandle.getStreamStateHandleID();
}

@Override
public StreamStateHandle getDelegateStateHandle() {
return delegateStateHandle;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.runtime.state;

import org.apache.flink.util.StringBasedID;

/**
* Unique ID that allows for physical comparison between state handles.
*
* <p>Different state objects (e.g. different files) representing the same piece of data must have
* different IDs (e.g. file names). This is different from {@link
* org.apache.flink.runtime.state.KeyedStateHandle#getStateHandleId} which returns the same ID.
*
* @see StateHandleID
*/
public class PhysicalStateHandleID extends StringBasedID {

private static final long serialVersionUID = 1L;

public PhysicalStateHandleID(String keyString) {
super(keyString);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,12 @@ public Optional<byte[]> asBytesIfInMemory() {
"This is only a placeholder to be replaced by a real StreamStateHandle in the checkpoint coordinator.");
}

@Override
public PhysicalStateHandleID getStreamStateHandleID() {
throw new UnsupportedOperationException(
"This is only a placeholder to be replaced by a real StreamStateHandle in the checkpoint coordinator.");
}

@Override
public void discardState() throws Exception {
// nothing to do.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,11 @@ public Optional<byte[]> asBytesIfInMemory() {
return wrappedStreamStateHandle.asBytesIfInMemory();
}

@Override
public PhysicalStateHandleID getStreamStateHandleID() {
return wrappedStreamStateHandle.getStreamStateHandleID();
}

@Override
public void discardState() throws Exception {
wrappedStreamStateHandle.discardState();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,4 +37,7 @@ public interface StreamStateHandle extends StateObject {

/** @return Content of this handle as bytes array if it is already in memory. */
Optional<byte[]> asBytesIfInMemory();

/** @return a unique identifier of this handle. */
PhysicalStateHandleID getStreamStateHandleID();
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.runtime.state.CheckpointBoundKeyedStateHandle;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.KeyedStateHandle;
import org.apache.flink.runtime.state.PhysicalStateHandleID;
import org.apache.flink.runtime.state.SharedStateRegistry;
import org.apache.flink.runtime.state.SharedStateRegistryKey;
import org.apache.flink.runtime.state.StateHandleID;
Expand Down Expand Up @@ -280,6 +281,11 @@ public Optional<byte[]> asBytesIfInMemory() {
throw new UnsupportedOperationException("Should not call here.");
}

@Override
public PhysicalStateHandleID getStreamStateHandleID() {
throw new UnsupportedOperationException("Should not call here.");
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.PhysicalStateHandleID;
import org.apache.flink.runtime.state.StreamStateHandle;

import java.io.IOException;
Expand All @@ -43,6 +44,8 @@ public class FileStateHandle implements StreamStateHandle {
/** The size of the state in the file. */
private final long stateSize;

private final PhysicalStateHandleID physicalID;

/**
* Creates a new file state for the given file path.
*
Expand All @@ -52,6 +55,7 @@ public FileStateHandle(Path filePath, long stateSize) {
checkArgument(stateSize >= -1);
this.filePath = checkNotNull(filePath);
this.stateSize = stateSize;
this.physicalID = new PhysicalStateHandleID(filePath.toUri().toString());
}

/**
Expand All @@ -73,6 +77,11 @@ public Optional<byte[]> asBytesIfInMemory() {
return Optional.empty();
}

@Override
public PhysicalStateHandleID getStreamStateHandleID() {
return physicalID;
}

/**
* Discard the state by deleting the file that stores the state. If the parent directory of the
* state is empty after deleting the state file, it is also deleted.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.runtime.state.memory;

import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.runtime.state.PhysicalStateHandleID;
import org.apache.flink.runtime.state.StreamStateHandle;
import org.apache.flink.util.Preconditions;

Expand All @@ -40,10 +41,13 @@ public class ByteStreamStateHandle implements StreamStateHandle {
*/
private final String handleName;

private final PhysicalStateHandleID physicalID;

/** Creates a new ByteStreamStateHandle containing the given data. */
public ByteStreamStateHandle(String handleName, byte[] data) {
this.handleName = Preconditions.checkNotNull(handleName);
this.data = Preconditions.checkNotNull(data);
this.physicalID = new PhysicalStateHandleID(handleName);
}

@Override
Expand All @@ -56,6 +60,11 @@ public Optional<byte[]> asBytesIfInMemory() {
return Optional.of(getData());
}

@Override
public PhysicalStateHandleID getStreamStateHandleID() {
return physicalID;
}

public byte[] getData() {
return data;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.StreamStateHandle;
import org.apache.flink.runtime.state.TestStreamStateHandle;

import org.junit.Test;

Expand Down Expand Up @@ -101,7 +101,7 @@ private static void testSerializabilityEqualsHashCode(Serializable o) throws IOE
assertNotNull(copy.toString());
}

private static class MyHandle implements StreamStateHandle {
private static class MyHandle implements TestStreamStateHandle {

private static final long serialVersionUID = 8128146204128728332L;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ public boolean equals(Object o) {
}

public static class ChangelogStateHandleWrapper extends InMemoryChangelogStateHandle
implements StreamStateHandle {
implements TestStreamStateHandle {
private static final long serialVersionUID = 1L;
private volatile boolean isDiscarded;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ public void testRegisterChangelogStateBackendHandles() throws InterruptedExcepti
assertFalse(nonMaterializedState2.isDiscarded());
}

private static class TestSharedState implements StreamStateHandle {
private static class TestSharedState implements TestStreamStateHandle {
private static final long serialVersionUID = 4468635881465159780L;

private SharedStateRegistryKey key;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
package org.apache.flink.runtime.state;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

/**
* Test {@link StreamStateHandle} that implements {@link #getStreamStateHandleID()} using {@link
* System#identityHashCode(Object)}.
*/
public interface TestStreamStateHandle extends StreamStateHandle {

default PhysicalStateHandleID getStreamStateHandleID() {
return new PhysicalStateHandleID(Integer.toString(System.identityHashCode(this)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
package org.apache.flink.runtime.state.testutils;

import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.runtime.state.StreamStateHandle;
import org.apache.flink.runtime.state.TestStreamStateHandle;

import java.io.IOException;
import java.util.Optional;
Expand All @@ -28,7 +28,7 @@
* A simple dummy implementation of a stream state handle that can be passed in tests. The handle
* cannot open an input stream.
*/
public class EmptyStreamStateHandle implements StreamStateHandle {
public class EmptyStreamStateHandle implements TestStreamStateHandle {

private static final long serialVersionUID = 0L;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.StateHandleID;
import org.apache.flink.runtime.state.StreamStateHandle;
import org.apache.flink.runtime.state.TestStreamStateHandle;
import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
import org.apache.flink.util.TestLogger;

Expand Down Expand Up @@ -137,7 +138,7 @@ private static class SpecifiedException extends IOException {
}
}

private static class ThrowingStateHandle implements StreamStateHandle {
private static class ThrowingStateHandle implements TestStreamStateHandle {
private static final long serialVersionUID = -2102069659550694805L;

private final IOException expectedException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
import org.apache.flink.runtime.state.OperatorStreamStateHandle;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.runtime.state.StreamStateHandle;
import org.apache.flink.runtime.state.TestStreamStateHandle;
import org.apache.flink.runtime.state.TestTaskStateManager;
import org.apache.flink.runtime.taskexecutor.KvStateService;
import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker;
Expand Down Expand Up @@ -299,7 +300,7 @@ private static Task createTask(
// ------------------------------------------------------------------------

@SuppressWarnings("serial")
private static class InterruptLockingStateHandle implements StreamStateHandle {
private static class InterruptLockingStateHandle implements TestStreamStateHandle {

private static final long serialVersionUID = 1L;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@
import org.apache.flink.runtime.state.OperatorStateBackend;
import org.apache.flink.runtime.state.OperatorStateHandle;
import org.apache.flink.runtime.state.OperatorStreamStateHandle;
import org.apache.flink.runtime.state.PhysicalStateHandleID;
import org.apache.flink.runtime.state.SharedStateRegistry;
import org.apache.flink.runtime.state.SnapshotResult;
import org.apache.flink.runtime.state.StateBackendFactory;
Expand Down Expand Up @@ -2400,6 +2401,11 @@ public FSDataInputStream openInputStream() throws IOException {
throw new IOException("Cannot open input streams in testing implementation.");
}

@Override
public PhysicalStateHandleID getStreamStateHandleID() {
throw new RuntimeException("Cannot return ID in testing implementation.");
}

@Override
public Optional<byte[]> asBytesIfInMemory() {
return Optional.empty();
Expand Down

0 comments on commit 5b58ef6

Please sign in to comment.