Thesharing commented on a change in pull request #16314:
URL: https://github.com/apache/flink/pull/16314#discussion_r672938170
##########
File path: flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
##########
@@ -618,6 +620,18 @@ public static void writeObjectToConfig(Object o,
Configuration config, String ke
}
}
+ public static <T> T uncompressAndDeserializeObject(byte[] bytes,
ClassLoader cl)
+ throws IOException, ClassNotFoundException {
+ return uncompressAndDeserializeObject(bytes, cl, false);
+ }
+
+ public static <T> T uncompressAndDeserializeObject(
Review comment:
> Seems this method is never used except for the other
`uncompressAndDeserializeObject`?
Yes, this method should be removed.
##########
File path:
flink-core/src/main/java/org/apache/flink/util/CompressedSerializedValue.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+
+/** An extension of {@link SerializedValue} that compresses the value after
the serialization. */
+public class CompressedSerializedValue<T> extends SerializedValue<T> {
+
+ private static final long serialVersionUID = -4358765382738374654L;
+
+ private CompressedSerializedValue(byte[] serializedData) {
Review comment:
> looks to me this constructor and the `fromBytes()` method are not used
in following commits.
It's used in `CompressedSerializeValueTest`, would it be okay to add an
annotation `@VisibleForTesting`?
##########
File path: flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
##########
@@ -618,6 +620,18 @@ public static void writeObjectToConfig(Object o,
Configuration config, String ke
}
}
+ public static <T> T uncompressAndDeserializeObject(byte[] bytes,
ClassLoader cl)
+ throws IOException, ClassNotFoundException {
+ return uncompressAndDeserializeObject(bytes, cl, false);
+ }
+
+ public static <T> T uncompressAndDeserializeObject(
+ byte[] bytes, ClassLoader cl, boolean isFailureTolerant)
+ throws IOException, ClassNotFoundException {
+ return deserializeObject(
+ new InflaterInputStream(new ByteArrayInputStream(bytes)), cl,
isFailureTolerant);
Review comment:
It seems that GZIPInputStream compresses data more slowly with higher
compression ratio. It suits better for large files that need to be archived.
Also it's less popular than InflaterInputStream. We can set up a comparison
test between these two.
##########
File path:
flink-core/src/main/java/org/apache/flink/util/CompressedSerializedValue.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+
+/** An extension of {@link SerializedValue} that compresses the value after
the serialization. */
+public class CompressedSerializedValue<T> extends SerializedValue<T> {
Review comment:
Sorry for missing this annotation.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java
##########
@@ -59,17 +62,34 @@
@Nonnegative private final int consumedSubpartitionIndex;
/** An input channel for each consumed subpartition. */
- private final ShuffleDescriptor[] inputChannels;
+ private transient ShuffleDescriptor[] inputChannels;
+
+ /** Serialized value of shuffle descriptors. */
+ private final CompressedSerializedValue<ShuffleDescriptor[]>
serializedInputChannels;
Review comment:
Thank you for pointing this out. I've changed it into `SerializedValue`.
##########
File path:
flink-core/src/test/java/org/apache/flink/util/CompressedSerializedValueTest.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.util;
+
+import org.apache.flink.core.testutils.CommonTestUtils;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+/** Tests for {@link CompressedSerializedValue}. */
+public class CompressedSerializedValueTest {
+ @Test
+ public void testSimpleValue() {
+ try {
+ final String value = "teststring";
+
+ CompressedSerializedValue<String> v =
CompressedSerializedValue.fromObject(value);
+ CompressedSerializedValue<String> copy =
CommonTestUtils.createCopySerializable(v);
+
+ assertEquals(value,
v.deserializeValue(getClass().getClassLoader()));
+ assertEquals(value,
copy.deserializeValue(getClass().getClassLoader()));
+
+ assertEquals(v, copy);
+ assertEquals(v.hashCode(), copy.hashCode());
+
+ assertNotNull(v.toString());
+ assertNotNull(copy.toString());
+
+ assertNotEquals(0, v.getSize());
+ assertArrayEquals(v.getByteArray(), copy.getByteArray());
+
+ byte[] bytes = v.getByteArray();
+ CompressedSerializedValue<String> saved =
+ CompressedSerializedValue.fromBytes(Arrays.copyOf(bytes,
bytes.length));
+ assertEquals(v, saved);
+ assertArrayEquals(v.getByteArray(), saved.getByteArray());
+ } catch (Exception e) {
+ e.printStackTrace();
Review comment:
Sorry for missing this. The `try-catch` has been removed.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
##########
@@ -52,6 +56,9 @@
private final ResultPartitionType resultType;
+ private final Map<ConsumedPartitionGroup,
CompressedSerializedValue<ShuffleDescriptor[]>>
Review comment:
Done.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraph.java
##########
@@ -1276,6 +1276,14 @@ private void maybeReleasePartitions(final Execution
attempt) {
private void releasePartitions(final List<IntermediateResultPartitionID>
releasablePartitions) {
if (releasablePartitions.size() > 0) {
+
+ // Remove cached ShuffleDescriptor when partition is released
Review comment:
`resetForNewExecution()` removes the cache when failover occurs.
`releasePartitions` removes the cache when the partition is released. This will
make sure when partitions are no longer valid, the cache related to the
partitions will be removed, too.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactoryTest.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.runtime.deployment;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.blob.BlobWriter;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import
org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition;
+import
org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import
org.apache.flink.runtime.executiongraph.failover.flip1.TestRestartBackoffTimeStrategy;
+import
org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphBuilder;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder;
+import org.apache.flink.runtime.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.scheduler.SchedulerTestingUtils;
+import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.util.CompressedSerializedValue;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/** Tests for {@link TaskDeploymentDescriptorFactory}. */
+public class TaskDeploymentDescriptorFactoryTest extends TestLogger {
+
+ private static final int PARALLELISM = 4;
+
+ public ExecutionJobVertex setupExecutionGraph(JobID jobId, BlobWriter
blobWriter)
+ throws JobException, JobExecutionException {
+ final JobVertex v1 = createJobVertex("v1", PARALLELISM);
+ final JobVertex v2 = createJobVertex("v2", PARALLELISM);
+
+ v2.connectNewDataSetAsInput(
+ v1, DistributionPattern.ALL_TO_ALL,
ResultPartitionType.BLOCKING);
+
+ final List<JobVertex> ordered = new ArrayList<>(Arrays.asList(v1, v2));
+
+ final ExecutionGraph executionGraph =
+ createExecutionGraphDirectly(jobId, ordered, blobWriter);
+
+ return executionGraph.getJobVertex(v2.getID());
+ }
+
+ @Test
+ public void testCacheShuffleDescriptorAsNonOffloaded() throws Exception {
+ final ExecutionJobVertex ejv = setupExecutionGraph(new JobID(), new
VoidBlobWriter());
+
+ assert ejv != null;
Review comment:
Sorry for missing this. Removed.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]