Skip to content

Commit

Permalink
[FLINK-5602] Introduce artifical namespace serializer for migration
Browse files Browse the repository at this point in the history
This closes apache#3200.
This closes apache#3198.
  • Loading branch information
StefanRRichter authored and joseprupi committed Feb 12, 2017
1 parent ea455d1 commit 13e9b93
Show file tree
Hide file tree
Showing 3 changed files with 134 additions and 16 deletions.
Expand Up @@ -34,6 +34,7 @@
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.MigrationNamespaceSerializerProxy;
import org.apache.flink.migration.MigrationUtil;
import org.apache.flink.migration.contrib.streaming.state.RocksDBStateBackend;
import org.apache.flink.runtime.io.async.AbstractAsyncIOCallable;
Expand Down Expand Up @@ -1155,15 +1156,16 @@ private void restoreOldSavepointKeyedState(Collection<KeyGroupsStateHandle> rest
columnFamilyMapping.put(mappingByte, stateDescriptor);

// this will fill in the k/v state information
getColumnFamily(stateDescriptor, null);
getColumnFamily(stateDescriptor, MigrationNamespaceSerializerProxy.INSTANCE);
}

// 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),null);
ColumnFamilyHandle handle = getColumnFamily(
columnFamilyMapping.get(mappingByte), MigrationNamespaceSerializerProxy.INSTANCE);

byte[] keyAndNamespace = BytePrimitiveArraySerializer.INSTANCE.deserialize(inputView);

Expand Down
@@ -0,0 +1,116 @@
/*
* 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://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;

import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;

import java.io.IOException;
import java.io.Serializable;

/**
* The purpose of this class is the be filled in as a placeholder for the namespace serializer when migrating from
* Flink 1.1 savepoint (which did not include the namespace serializer) to Flink 1.2 (which always must include a
* (non-null) namespace serializer. This is then replaced as soon as the user is re-registering her state again for
* the first run under Flink 1.2 and provides again the real namespace serializer.
*/
@Deprecated
public class MigrationNamespaceSerializerProxy extends TypeSerializer<Serializable> {

public static final MigrationNamespaceSerializerProxy INSTANCE = new MigrationNamespaceSerializerProxy();

private static final long serialVersionUID = -707800010807094491L;

private MigrationNamespaceSerializerProxy() {
}

@Override
public boolean isImmutableType() {
return false;
}

@Override
public TypeSerializer<Serializable> duplicate() {
throw new UnsupportedOperationException(
"This is just a proxy used during migration until the real type serializer is provided by the user.");
}

@Override
public Serializable createInstance() {
throw new UnsupportedOperationException(
"This is just a proxy used during migration until the real type serializer is provided by the user.");
}

@Override
public Serializable copy(Serializable from) {
throw new UnsupportedOperationException(
"This is just a proxy used during migration until the real type serializer is provided by the user.");
}

@Override
public Serializable copy(Serializable from, Serializable reuse) {
throw new UnsupportedOperationException(
"This is just a proxy used during migration until the real type serializer is provided by the user.");
}

@Override
public int getLength() {
return -1;
}

@Override
public void serialize(Serializable record, DataOutputView target) throws IOException {
throw new UnsupportedOperationException(
"This is just a proxy used during migration until the real type serializer is provided by the user.");
}

@Override
public Serializable deserialize(DataInputView source) throws IOException {
throw new UnsupportedOperationException(
"This is just a proxy used during migration until the real type serializer is provided by the user.");
}

@Override
public Serializable deserialize(Serializable reuse, DataInputView source) throws IOException {
throw new UnsupportedOperationException(
"This is just a proxy used during migration until the real type serializer is provided by the user.");
}

@Override
public void copy(DataInputView source, DataOutputView target) throws IOException {
throw new UnsupportedOperationException(
"This is just a proxy used during migration until the real type serializer is provided by the user.");
}

@Override
public boolean equals(Object obj) {
return obj instanceof MigrationNamespaceSerializerProxy;
}

@Override
public boolean canEqual(Object obj) {
return true;
}

@Override
public int hashCode() {
return 42;
}
}
Expand Up @@ -20,6 +20,7 @@

import org.apache.flink.api.common.state.StateDescriptor;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.migration.MigrationNamespaceSerializerProxy;

import static org.apache.flink.util.Preconditions.checkNotNull;

Expand All @@ -38,10 +39,11 @@ public class RegisteredBackendStateMetaInfo<N, S> {
private final TypeSerializer<S> stateSerializer;

public RegisteredBackendStateMetaInfo(KeyedBackendSerializationProxy.StateMetaInfo<N, S> metaInfoProxy) {
this.stateType = metaInfoProxy.getStateType();
this.name = metaInfoProxy.getStateName();
this.namespaceSerializer = metaInfoProxy.getNamespaceSerializerSerializationProxy().getTypeSerializer();
this.stateSerializer = metaInfoProxy.getStateSerializerSerializationProxy().getTypeSerializer();
this(
metaInfoProxy.getStateType(),
metaInfoProxy.getStateName(),
metaInfoProxy.getNamespaceSerializerSerializationProxy().getTypeSerializer(),
metaInfoProxy.getStateSerializerSerializationProxy().getTypeSerializer());
}

public RegisteredBackendStateMetaInfo(
Expand All @@ -52,8 +54,8 @@ public RegisteredBackendStateMetaInfo(

this.stateType = checkNotNull(stateType);
this.name = checkNotNull(name);
this.namespaceSerializer = namespaceSerializer;
this.stateSerializer = stateSerializer;
this.namespaceSerializer = checkNotNull(namespaceSerializer);
this.stateSerializer = checkNotNull(stateSerializer);
}

public StateDescriptor.Type getStateType() {
Expand Down Expand Up @@ -92,10 +94,10 @@ public boolean isCompatibleWith(RegisteredBackendStateMetaInfo<?, ?> other) {
return false;
}

return ((namespaceSerializer == null && other.namespaceSerializer == null)
|| namespaceSerializer == null || other.namespaceSerializer == null
|| namespaceSerializer.isCompatibleWith(other.namespaceSerializer))
&& stateSerializer.isCompatibleWith(other.stateSerializer);
return (stateSerializer.isCompatibleWith(other.stateSerializer)) &&
(namespaceSerializer.isCompatibleWith(other.namespaceSerializer)
// we also check if there is just a migration proxy that should be replaced by any real serializer
|| other.namespaceSerializer instanceof MigrationNamespaceSerializerProxy);
}

@Override
Expand All @@ -118,10 +120,8 @@ public boolean equals(Object o) {
return false;
}

if (getNamespaceSerializer() != null ? !getNamespaceSerializer().equals(that.getNamespaceSerializer()) : that.getNamespaceSerializer() != null) {
return false;
}
return getStateSerializer() != null ? getStateSerializer().equals(that.getStateSerializer()) : that.getStateSerializer() == null;
return getStateSerializer().equals(that.getStateSerializer())
&& getNamespaceSerializer().equals(that.getNamespaceSerializer());
}

@Override
Expand Down

0 comments on commit 13e9b93

Please sign in to comment.