This repository has been archived by the owner on Dec 14, 2022. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 120
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Browse files
Browse the repository at this point in the history
(cherry picked from commit 68b5260)
- Loading branch information
1 parent
d6e2737
commit a8f9665
Showing
5 changed files
with
153 additions
and
18 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
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
59 changes: 59 additions & 0 deletions
59
...n/java/org/apache/flink/streaming/util/serialization/ThreadSafeDeserializationSchema.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 | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,59 @@ | ||
/* | ||
* Licensed 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.streaming.util.serialization; | ||
|
||
import org.apache.flink.api.common.serialization.DeserializationSchema; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
|
||
import java.io.IOException; | ||
|
||
/** | ||
* Because the Pulsar Source is designed to be multi-threaded, | ||
* Flink's internal design of the Source is single-threaded, | ||
* so, DeserializationSchema instances are oriented to single-threaded, | ||
* and thread safety issues exist when they are accessed by multiple threads at the same time. Cause the message deserialization to fail. | ||
*/ | ||
public class ThreadSafeDeserializationSchema<T> implements DeserializationSchema<T> { | ||
|
||
private DeserializationSchema<T> deserializationSchema; | ||
|
||
private ThreadSafeDeserializationSchema(DeserializationSchema<T> deserializationSchema) { | ||
this.deserializationSchema = deserializationSchema; | ||
} | ||
|
||
public static ThreadSafeDeserializationSchema of(DeserializationSchema deserializationSchema) { | ||
return deserializationSchema != null ? new ThreadSafeDeserializationSchema(deserializationSchema) : null; | ||
} | ||
|
||
@Override | ||
public synchronized void open(InitializationContext context) throws Exception { | ||
deserializationSchema.open(context); | ||
} | ||
|
||
@Override | ||
public synchronized T deserialize(byte[] bytes) throws IOException { | ||
return deserializationSchema.deserialize(bytes); | ||
} | ||
|
||
@Override | ||
public synchronized boolean isEndOfStream(T object) { | ||
return deserializationSchema.isEndOfStream(object); | ||
} | ||
|
||
@Override | ||
public synchronized TypeInformation getProducedType() { | ||
return deserializationSchema.getProducedType(); | ||
} | ||
} |
85 changes: 85 additions & 0 deletions
85
...va/org/apache/flink/streaming/util/serialization/ThreadSafeDeserializationSchemaTest.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 | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
/* | ||
* Licensed 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.streaming.util.serialization; | ||
|
||
import org.apache.flink.api.common.serialization.DeserializationSchema; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
|
||
import org.junit.Assert; | ||
import org.junit.Test; | ||
|
||
import java.io.IOException; | ||
|
||
/** | ||
* thread safe {@link ThreadSafeDeserializationSchema} test. | ||
*/ | ||
public class ThreadSafeDeserializationSchemaTest { | ||
|
||
@Test | ||
public void deserialize() throws InterruptedException { | ||
NoThreadSafeDeserializationSchema noThreadSafeDeserializationSchema = new NoThreadSafeDeserializationSchema(); | ||
DeserializationSchema deserializationSchema = ThreadSafeDeserializationSchema.of(noThreadSafeDeserializationSchema); | ||
Thread[] threads = new Thread[10]; | ||
for (int i = 0; i < 10; i++) { | ||
threads[i] = new Thread(() -> { | ||
try { | ||
for (int j = 0; j < 100; j++) { | ||
deserializationSchema.deserialize(null); | ||
} | ||
} catch (IOException e) { | ||
} | ||
}); | ||
threads[i].start(); | ||
} | ||
|
||
for (int i = 0; i < 10; i++) { | ||
threads[i].join(); | ||
} | ||
Assert.assertEquals(noThreadSafeDeserializationSchema.getCount(), 1000); | ||
} | ||
|
||
class NoThreadSafeDeserializationSchema implements DeserializationSchema { | ||
|
||
private int count = 0; | ||
private int tmpCount = 0; | ||
|
||
public int getCount() { | ||
return count; | ||
} | ||
|
||
@Override | ||
public Object deserialize(byte[] bytes) throws IOException { | ||
tmpCount = count; | ||
try { | ||
Thread.sleep(1); | ||
} catch (InterruptedException e) { | ||
e.printStackTrace(); | ||
} | ||
tmpCount++; | ||
count = tmpCount; | ||
return null; | ||
} | ||
|
||
@Override | ||
public boolean isEndOfStream(Object o) { | ||
return false; | ||
} | ||
|
||
@Override | ||
public TypeInformation getProducedType() { | ||
return null; | ||
} | ||
} | ||
} |