Skip to content

Commit

Permalink
[FLINK-5041] Savepoint backwards compatibility 1.1 -> 1.2
Browse files Browse the repository at this point in the history
This addresses Savepoint, TaskState, StateHandels, KeyedStateBackends.

This closes apache#2781.
  • Loading branch information
StefanRRichter authored and uce committed Dec 7, 2016
1 parent a6e80da commit af3bf83
Show file tree
Hide file tree
Showing 57 changed files with 3,823 additions and 227 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,18 +31,23 @@
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataInputViewStreamWrapper;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
import org.apache.flink.migration.MigrationUtil;
import org.apache.flink.migration.contrib.streaming.state.RocksDBStateBackend;
import org.apache.flink.runtime.io.async.AbstractAsyncIOCallable;
import org.apache.flink.runtime.io.async.AsyncStoppableTaskWithCallback;
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.runtime.state.DoneFuture;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
import org.apache.flink.runtime.state.KeyGroupsStateHandle;
import org.apache.flink.runtime.state.StreamStateHandle;
Expand All @@ -62,8 +67,10 @@
import org.slf4j.LoggerFactory;

import java.io.Closeable;
import java.io.EOFException;
import java.io.File;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
Expand Down Expand Up @@ -205,8 +212,13 @@ public RocksDBKeyedStateBackend(
}

try {
RocksDBRestoreOperation restoreOperation = new RocksDBRestoreOperation(this);
restoreOperation.doRestore(restoreState);
if (MigrationUtil.isOldSavepointKeyedState(restoreState)) {
LOG.info("Converting RocksDB state from old savepoint.");
restoreOldSavepointKeyedState(restoreState);
} else {
RocksDBRestoreOperation restoreOperation = new RocksDBRestoreOperation(this);
restoreOperation.doRestore(restoreState);
}
} catch (Exception ex) {
dispose();
throw ex;
Expand Down Expand Up @@ -1068,4 +1080,84 @@ public void close() {
public File getInstanceBasePath() {
return instanceBasePath;
}

/**
* For backwards compatibility, remove again later!
*/
@Deprecated
private void restoreOldSavepointKeyedState(Collection<KeyGroupsStateHandle> restoreState) throws Exception {

if (restoreState.isEmpty()) {
return;
}

Preconditions.checkState(1 == restoreState.size(), "Only one element expected here.");
HashMap<String, RocksDBStateBackend.FinalFullyAsyncSnapshot> namedStates =
InstantiationUtil.deserializeObject(restoreState.iterator().next().openInputStream(), userCodeClassLoader);

Preconditions.checkState(1 == namedStates.size(), "Only one element expected here.");
DataInputView inputView = namedStates.values().iterator().next().stateHandle.getState(userCodeClassLoader);

// clear k/v state information before filling it
kvStateInformation.clear();

// first get the column family mapping
int numColumns = inputView.readInt();
Map<Byte, StateDescriptor> columnFamilyMapping = new HashMap<>(numColumns);
for (int i = 0; i < numColumns; i++) {
byte mappingByte = inputView.readByte();

ObjectInputStream ooIn =
new InstantiationUtil.ClassLoaderObjectInputStream(
new DataInputViewStream(inputView), userCodeClassLoader);

StateDescriptor stateDescriptor = (StateDescriptor) ooIn.readObject();

columnFamilyMapping.put(mappingByte, stateDescriptor);

// this will fill in the k/v state information
getColumnFamily(stateDescriptor);
}

// try and read until EOF
try {
// the EOFException will get us out of this...
while (true) {
byte mappingByte = inputView.readByte();
ColumnFamilyHandle handle = getColumnFamily(columnFamilyMapping.get(mappingByte));
byte[] keyAndNamespace = BytePrimitiveArraySerializer.INSTANCE.deserialize(inputView);

ByteArrayInputStreamWithPos bis = new ByteArrayInputStreamWithPos(keyAndNamespace);

K reconstructedKey = keySerializer.deserialize(new DataInputViewStreamWrapper(bis));
int len = bis.getPosition();

int keyGroup = (byte) KeyGroupRangeAssignment.assignToKeyGroup(reconstructedKey, numberOfKeyGroups);

if (keyGroupPrefixBytes == 1) {
// copy and override one byte (42) between key and namespace
System.arraycopy(keyAndNamespace, 0, keyAndNamespace, 1, len);
keyAndNamespace[0] = (byte) keyGroup;
} else {
byte[] largerKey = new byte[1 + keyAndNamespace.length];

// write key-group
largerKey[0] = (byte) ((keyGroup >> 8) & 0xFF);
largerKey[1] = (byte) (keyGroup & 0xFF);

// write key
System.arraycopy(keyAndNamespace, 0, largerKey, 2, len);

//skip one byte (42), write namespace
System.arraycopy(keyAndNamespace, 1 + len, largerKey, 2 + len, keyAndNamespace.length - len - 1);
keyAndNamespace = largerKey;
}

byte[] value = BytePrimitiveArraySerializer.INSTANCE.deserialize(inputView);
db.put(handle, keyAndNamespace, value);
}
} catch (EOFException e) {
// expected
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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.migration.contrib.streaming.state;

import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.migration.runtime.state.AbstractStateBackend;
import org.apache.flink.migration.runtime.state.KvStateSnapshot;
import org.apache.flink.migration.runtime.state.StateHandle;

import java.io.IOException;

import static java.util.Objects.requireNonNull;

@Deprecated
public class RocksDBStateBackend extends AbstractStateBackend {
private static final long serialVersionUID = 1L;

/**
* Dummy {@link KvStateSnapshot} that holds the state of our one RocksDB data base.
*/
public static class FinalFullyAsyncSnapshot implements KvStateSnapshot<Object, Object, ValueState<Object>, ValueStateDescriptor<Object>> {
private static final long serialVersionUID = 1L;

public final StateHandle<DataInputView> stateHandle;
final long checkpointId;

/**
* Creates a new snapshot from the given state parameters.
*/
private FinalFullyAsyncSnapshot(StateHandle<DataInputView> stateHandle, long checkpointId) {
this.stateHandle = requireNonNull(stateHandle);
this.checkpointId = checkpointId;
}

@Override
public final void discardState() throws Exception {
stateHandle.discardState();
}

@Override
public final long getStateSize() throws Exception {
return stateHandle.getStateSize();
}

@Override
public void close() throws IOException {
stateHandle.close();
}
}
}
2 changes: 1 addition & 1 deletion flink-core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ under the License.
<artifactId>joda-convert</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</dependencies>

<build>
<plugins>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
/*
* 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.core.fs;

import org.apache.flink.util.IOUtils;
import org.apache.flink.util.Preconditions;

import java.io.EOFException;
import java.io.IOException;

/**
* Abstract base class for wrappers over multiple {@link FSDataInputStream}, which gives a contiguous view on all inner
* streams and makes them look like a single stream, in which we can read, seek, etc.
*/
public abstract class AbstractMultiFSDataInputStream extends FSDataInputStream {

/** Inner stream for the currently accessed segment of the virtual global stream */
protected FSDataInputStream delegate;

/** Position in the virtual global stream */
protected long totalPos;

/** Total available bytes in the virtual global stream */
protected long totalAvailable;

public AbstractMultiFSDataInputStream() {
this.totalPos = 0L;
}

@Override
public void seek(long desired) throws IOException {

if(desired == totalPos) {
return;
}

Preconditions.checkArgument(desired >= 0L);

if (desired > totalAvailable) {
throw new EOFException();
}

IOUtils.closeQuietly(delegate);
delegate = getSeekedStreamForOffset(desired);

this.totalPos = desired;
}

@Override
public long getPos() throws IOException {
return totalPos;
}

@Override
public int read() throws IOException {

if (null == delegate) {
return -1;
}

int val = delegate.read();

if (-1 == val) {
IOUtils.closeQuietly(delegate);
if (totalPos < totalAvailable) {
delegate = getSeekedStreamForOffset(totalPos);
} else {
delegate = null;
}
return read();
}

++totalPos;
return val;
}

@Override
public void close() throws IOException {
IOUtils.closeQuietly(delegate);
}

@Override
public long skip(long n) throws IOException {
seek(totalPos + n);
return n;
}

/**
* Delivers a the right stream for the given global stream offset. The returned stream is already seeked to the
* right local offset that correctly reflects the global offset.
*
* @param globalStreamOffset the global offset to which we seek
* @return a sub-stream, seeked to the correct local offset w.r.t. the global offset.
* @throws IOException
*/
protected abstract FSDataInputStream getSeekedStreamForOffset(long globalStreamOffset) throws IOException;
}
Loading

0 comments on commit af3bf83

Please sign in to comment.