Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-7826][QS] Add support for all types of state to the QS Client.
- Loading branch information
Showing
21 changed files
with
1,817 additions
and
188 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
71 changes: 71 additions & 0 deletions
71
...src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,71 @@ | |||
/* | |||
* 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.queryablestate.client.state; | |||
|
|||
import org.apache.flink.annotation.PublicEvolving; | |||
import org.apache.flink.api.common.state.AggregatingState; | |||
import org.apache.flink.api.common.state.AggregatingStateDescriptor; | |||
import org.apache.flink.runtime.query.netty.message.KvStateSerializer; | |||
import org.apache.flink.util.Preconditions; | |||
|
|||
import java.io.IOException; | |||
|
|||
/** | |||
* A read-only {@link AggregatingState} that <b>does not</b> allow for modifications. | |||
* | |||
* <p>This is the type of the result returned when querying Flink's keyed state using the | |||
* {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and | |||
* providing an {@link AggregatingStateDescriptor}. | |||
*/ | |||
@PublicEvolving | |||
public final class ImmutableAggregatingState<IN, OUT> extends ImmutableState implements AggregatingState<IN, OUT> { | |||
|
|||
private final OUT value; | |||
|
|||
private ImmutableAggregatingState(OUT value) { | |||
this.value = Preconditions.checkNotNull(value); | |||
} | |||
|
|||
@Override | |||
public OUT get() { | |||
return value; | |||
} | |||
|
|||
@Override | |||
public void add(Object newValue) { | |||
throw MODIFICATION_ATTEMPT_ERROR; | |||
} | |||
|
|||
@Override | |||
public void clear() { | |||
throw MODIFICATION_ATTEMPT_ERROR; | |||
} | |||
|
|||
public static <IN, ACC, OUT> ImmutableAggregatingState<IN, OUT> createState( | |||
final AggregatingStateDescriptor<IN, ACC, OUT> stateDescriptor, | |||
final byte[] serializedValue) throws IOException { | |||
|
|||
final ACC accumulator = KvStateSerializer.deserializeValue( | |||
serializedValue, | |||
stateDescriptor.getSerializer()); | |||
|
|||
final OUT state = stateDescriptor.getAggregateFunction().getResult(accumulator); | |||
return new ImmutableAggregatingState<>(state); | |||
} | |||
} |
70 changes: 70 additions & 0 deletions
70
...ava/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,70 @@ | |||
/* | |||
* 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.queryablestate.client.state; | |||
|
|||
import org.apache.flink.annotation.PublicEvolving; | |||
import org.apache.flink.api.common.state.FoldingState; | |||
import org.apache.flink.api.common.state.FoldingStateDescriptor; | |||
import org.apache.flink.runtime.query.netty.message.KvStateSerializer; | |||
import org.apache.flink.util.Preconditions; | |||
|
|||
import java.io.IOException; | |||
|
|||
/** | |||
* A read-only {@link FoldingState} that does not allow for modifications. | |||
* | |||
* <p>This is the result returned when querying Flink's keyed state using the | |||
* {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and | |||
* providing an {@link FoldingStateDescriptor}. | |||
*/ | |||
@PublicEvolving | |||
@Deprecated | |||
public final class ImmutableFoldingState<IN, ACC> extends ImmutableState implements FoldingState<IN, ACC> { | |||
|
|||
private final ACC value; | |||
|
|||
private ImmutableFoldingState(ACC value) { | |||
this.value = Preconditions.checkNotNull(value); | |||
} | |||
|
|||
@Override | |||
public ACC get() { | |||
return value; | |||
} | |||
|
|||
@Override | |||
public void add(Object newValue) { | |||
throw MODIFICATION_ATTEMPT_ERROR; | |||
} | |||
|
|||
@Override | |||
public void clear() { | |||
throw MODIFICATION_ATTEMPT_ERROR; | |||
} | |||
|
|||
public static <IN, ACC> ImmutableFoldingState<IN, ACC> createState( | |||
final FoldingStateDescriptor<IN, ACC> stateDescriptor, | |||
final byte[] serializedState) throws IOException { | |||
|
|||
final ACC state = KvStateSerializer.deserializeValue( | |||
serializedState, | |||
stateDescriptor.getSerializer()); | |||
return new ImmutableFoldingState<>(state); | |||
} | |||
} |
70 changes: 70 additions & 0 deletions
70
...e-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,70 @@ | |||
/* | |||
* 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.queryablestate.client.state; | |||
|
|||
import org.apache.flink.annotation.PublicEvolving; | |||
import org.apache.flink.api.common.state.ListState; | |||
import org.apache.flink.api.common.state.ListStateDescriptor; | |||
import org.apache.flink.runtime.query.netty.message.KvStateSerializer; | |||
import org.apache.flink.util.Preconditions; | |||
|
|||
import java.io.IOException; | |||
import java.util.List; | |||
|
|||
/** | |||
* A read-only {@link ListState} that does not allow for modifications. | |||
* | |||
* <p>This is the result returned when querying Flink's keyed state using the | |||
* {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and | |||
* providing an {@link ListStateDescriptor}. | |||
*/ | |||
@PublicEvolving | |||
public final class ImmutableListState<V> extends ImmutableState implements ListState<V> { | |||
|
|||
private final List<V> listState; | |||
|
|||
private ImmutableListState(final List<V> state) { | |||
this.listState = Preconditions.checkNotNull(state); | |||
} | |||
|
|||
@Override | |||
public Iterable<V> get() { | |||
return listState; | |||
} | |||
|
|||
@Override | |||
public void add(V value) { | |||
throw MODIFICATION_ATTEMPT_ERROR; | |||
} | |||
|
|||
@Override | |||
public void clear() { | |||
throw MODIFICATION_ATTEMPT_ERROR; | |||
} | |||
|
|||
public static <V> ImmutableListState<V> createState( | |||
final ListStateDescriptor<V> stateDescriptor, | |||
final byte[] serializedState) throws IOException { | |||
|
|||
final List<V> state = KvStateSerializer.deserializeList( | |||
serializedState, | |||
stateDescriptor.getElementSerializer()); | |||
return new ImmutableListState<>(state); | |||
} | |||
} |
Oops, something went wrong.