Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
8eadd3ec
F
flink
项目概览
doujutun3207
/
flink
与 Fork 源项目一致
从无法访问的项目Fork
通知
24
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
F
flink
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
提交
8eadd3ec
编写于
11月 17, 2014
作者:
T
Till Rohrmann
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
Removed JobStatusListener and ExecutionListener. Fixed LocalExecutor output for maven verify.
上级
c175ebe8
变更
21
展开全部
隐藏空白更改
内联
并排
Showing
21 changed file
with
72 addition
and
194 deletion
+72
-194
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
...ain/java/org/apache/flink/streaming/util/ClusterUtil.java
+0
-1
flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/Client.java
...link-yarn/src/main/java/org/apache/flink/yarn/Client.java
+7
-2
flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
.../main/scala/org/apache/flink/yarn/ApplicationClient.scala
+2
-2
flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
.../main/scala/org/apache/flink/yarn/ApplicationMaster.scala
+1
-1
flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
.../src/main/java/org/apache/flink/client/LocalExecutor.java
+7
-1
flink-dist/src/main/flink-bin/bin/start-cluster.sh
flink-dist/src/main/flink-bin/bin/start-cluster.sh
+1
-1
flink-dist/src/main/flink-bin/bin/start-local.sh
flink-dist/src/main/flink-bin/bin/start-local.sh
+1
-1
flink-dist/src/main/flink-bin/bin/stop-cluster.sh
flink-dist/src/main/flink-bin/bin/stop-cluster.sh
+1
-1
flink-dist/src/main/flink-bin/bin/stop-local.sh
flink-dist/src/main/flink-bin/bin/stop-local.sh
+1
-1
flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
...c/main/java/org/apache/flink/runtime/blob/BlobClient.java
+5
-1
flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
...c/main/java/org/apache/flink/runtime/blob/BlobServer.java
+12
-6
flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
...org/apache/flink/runtime/execution/ExecutionListener.java
+0
-33
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
...g/apache/flink/runtime/executiongraph/ExecutionGraph.java
+0
-40
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusListener.java
...pache/flink/runtime/executiongraph/JobStatusListener.java
+0
-36
flink-runtime/src/main/java/org/apache/flink/runtime/profiling/TaskManagerProfiler.java
...g/apache/flink/runtime/profiling/TaskManagerProfiler.java
+7
-6
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
.../main/java/org/apache/flink/runtime/taskmanager/Task.java
+2
-29
flink-runtime/src/main/resources/web-docs-infoserver/js/jquery-1.10.2.js
...rc/main/resources/web-docs-infoserver/js/jquery-1.10.2.js
+0
-6
flink-runtime/src/main/resources/web-docs-infoserver/js/jquery-2.1.0.js
...src/main/resources/web-docs-infoserver/js/jquery-2.1.0.js
+0
-0
flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
...la/org/apache/flink/runtime/taskmanager/TaskManager.scala
+1
-5
flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java
...ache/flink/test/clients/examples/LocalExecutorITCase.java
+1
-0
flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
...rative/DeltaIterationNotDependingOnSolutionSetITCase.java
+23
-21
未找到文件。
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
浏览文件 @
8eadd3ec
...
...
@@ -19,7 +19,6 @@ package org.apache.flink.streaming.util;
import
java.net.InetSocketAddress
;
import
org.apache.flink.api.java.ExecutionEnvironment
;
import
org.apache.flink.client.program.Client
;
import
org.apache.flink.client.program.ProgramInvocationException
;
import
org.apache.flink.configuration.ConfigConstants
;
...
...
flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/Client.java
浏览文件 @
8eadd3ec
...
...
@@ -29,6 +29,7 @@ import java.util.Collections;
import
java.util.HashMap
;
import
java.util.List
;
import
java.util.Map
;
import
java.util.concurrent.TimeUnit
;
import
java.util.jar.JarFile
;
import
akka.actor.ActorRef
;
...
...
@@ -71,6 +72,7 @@ import org.apache.hadoop.yarn.client.api.YarnClient;
import
org.apache.hadoop.yarn.client.api.YarnClientApplication
;
import
org.apache.hadoop.yarn.exceptions.YarnException
;
import
org.apache.hadoop.yarn.util.Records
;
import
scala.concurrent.duration.FiniteDuration
;
/**
* All classes in this package contain code taken from
...
...
@@ -328,6 +330,9 @@ public class Client {
LOG
.
warn
(
"Unable to find job manager port in configuration!"
);
jmPort
=
ConfigConstants
.
DEFAULT_JOB_MANAGER_IPC_PORT
;
}
FiniteDuration
timeout
=
new
FiniteDuration
(
GlobalConfiguration
.
getInteger
(
ConfigConstants
.
AKKA_ASK_TIMEOUT
,
ConfigConstants
.
DEFAULT_AKKA_ASK_TIMEOUT
),
TimeUnit
.
SECONDS
);
conf
=
Utils
.
initializeYarnConfiguration
();
...
...
@@ -520,7 +525,6 @@ public class Client {
// file that we write into the conf/ dir containing the jobManager address and the dop.
yarnPropertiesFile
=
new
File
(
confDirPath
+
CliFrontend
.
YARN_PROPERTIES_FILE
);
LOG
.
info
(
"Submitting application master "
+
appId
);
yarnClient
.
submitApplication
(
appContext
);
...
...
@@ -533,7 +537,8 @@ public class Client {
// start application client
LOG
.
info
(
"Start application client."
);
applicationClient
=
actorSystem
.
actorOf
(
Props
.
create
(
ApplicationClient
.
class
,
appId
,
jmPort
,
yarnClient
,
confDirPath
,
slots
,
taskManagerCount
,
dynamicPropertiesEncoded
));
yarnClient
,
confDirPath
,
slots
,
taskManagerCount
,
dynamicPropertiesEncoded
,
timeout
));
actorSystem
.
awaitTermination
();
...
...
flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
浏览文件 @
8eadd3ec
...
...
@@ -35,7 +35,7 @@ import scala.concurrent.duration._
class
ApplicationClient
(
appId
:
ApplicationId
,
port
:
Int
,
yarnClient
:
YarnClient
,
confDirPath
:
String
,
slots
:
Int
,
numTaskManagers
:
Int
,
dynamicPropertiesEncoded
:
String
)
dynamicPropertiesEncoded
:
String
,
timeout
:
FiniteDuration
)
extends
Actor
with
Consumer
with
ActorLogMessages
with
ActorLogging
{
import
context._
...
...
@@ -85,7 +85,7 @@ class ApplicationClient(appId: ApplicationId, port: Int, yarnClient: YarnClient,
writeYarnProperties
(
address
)
jobManager
=
Some
(
AkkaUtils
.
getReference
(
JobManager
.
getAkkaURL
(
address
)))
jobManager
=
Some
(
AkkaUtils
.
getReference
(
JobManager
.
getAkkaURL
(
address
))
(
system
,
timeout
)
)
jobManager
.
get
!
RegisterMessageListener
pollingTimer
foreach
{
...
...
flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
浏览文件 @
8eadd3ec
...
...
@@ -159,7 +159,7 @@ object ApplicationMaster{
val
args
=
Array
[
String
](
"--configDir"
,
pathToConfig
)
LOG
.
info
(
s
"Config path: ${pathToConfig}."
)
val
(
hostname
,
port
,
configuration
)
=
JobManager
.
parseArgs
(
args
)
val
(
hostname
,
port
,
configuration
,
_
)
=
JobManager
.
parseArgs
(
args
)
implicit
val
jobManagerSystem
=
YarnUtils
.
createActorSystem
(
hostname
,
port
,
configuration
)
...
...
flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
浏览文件 @
8eadd3ec
...
...
@@ -60,6 +60,8 @@ public class LocalExecutor extends PlanExecutor {
private
int
taskManagerNumSlots
=
DEFAULT_TASK_MANAGER_NUM_SLOTS
;
private
boolean
defaultOverwriteFiles
=
DEFAULT_OVERWRITE
;
private
boolean
printStatusDuringExecution
=
true
;
// --------------------------------------------------------------------------------------------
...
...
@@ -82,6 +84,10 @@ public class LocalExecutor extends PlanExecutor {
public
void
setTaskManagerNumSlots
(
int
taskManagerNumSlots
)
{
this
.
taskManagerNumSlots
=
taskManagerNumSlots
;
}
public
int
getTaskManagerNumSlots
()
{
return
this
.
taskManagerNumSlots
;
}
public
void
setPrintStatusDuringExecution
(
boolean
printStatus
)
{
this
.
printStatusDuringExecution
=
printStatus
;
}
// --------------------------------------------------------------------------------------------
...
...
@@ -164,7 +170,7 @@ public class LocalExecutor extends PlanExecutor {
ActorRef
jobClient
=
flink
.
getJobClient
();
return
JobClient
.
submitJobAndWait
(
jobGraph
,
true
,
jobClient
);
return
JobClient
.
submitJobAndWait
(
jobGraph
,
printStatusDuringExecution
,
jobClient
);
}
finally
{
if
(
shutDownAtEnd
)
{
...
...
flink-dist/src/main/flink-bin/bin/start-cluster.sh
浏览文件 @
8eadd3ec
...
...
@@ -35,7 +35,7 @@ if [ ! -f "$HOSTLIST" ]; then
fi
# cluster mode, bring up job manager locally and a task manager on every slave host
"
$FLINK_BIN_DIR
"
/job
M
anager.sh start cluster
"
$FLINK_BIN_DIR
"
/job
m
anager.sh start cluster
GOON
=
true
while
$GOON
...
...
flink-dist/src/main/flink-bin/bin/start-local.sh
浏览文件 @
8eadd3ec
...
...
@@ -24,4 +24,4 @@ bin=`cd "$bin"; pwd`
.
"
$bin
"
/config.sh
# local mode, only bring up job manager. The job manager will start an internal task manager
"
$FLINK_BIN_DIR
"
/job
M
anager.sh start
local
"
$FLINK_BIN_DIR
"
/job
m
anager.sh start
local
flink-dist/src/main/flink-bin/bin/stop-cluster.sh
浏览文件 @
8eadd3ec
...
...
@@ -46,4 +46,4 @@ do
done
<
$HOSTLIST
# cluster mode, stop the job manager locally and stop the task manager on every slave host
"
$FLINK_BIN_DIR
"
/job
M
anager.sh stop
"
$FLINK_BIN_DIR
"
/job
m
anager.sh stop
flink-dist/src/main/flink-bin/bin/stop-local.sh
浏览文件 @
8eadd3ec
...
...
@@ -24,4 +24,4 @@ bin=`cd "$bin"; pwd`
.
"
$bin
"
/config.sh
# stop local job manager (has an internal task manager)
"
$FLINK_BIN_DIR
"
/job
M
anager.sh stop
"
$FLINK_BIN_DIR
"
/job
m
anager.sh stop
flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
浏览文件 @
8eadd3ec
...
...
@@ -55,7 +55,11 @@ public final class BlobClient implements Closeable {
public
BlobClient
(
final
InetSocketAddress
serverAddress
)
throws
IOException
{
this
.
socket
=
new
Socket
();
this
.
socket
.
connect
(
serverAddress
);
try
{
this
.
socket
.
connect
(
serverAddress
);
}
catch
(
IOException
e
){
throw
new
IOException
(
"Could not connect to BlobServer at address "
+
serverAddress
,
e
);
}
}
/**
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
浏览文件 @
8eadd3ec
...
...
@@ -101,15 +101,21 @@ public final class BlobServer extends Thread implements BlobService{
*/
public
BlobServer
()
throws
IOException
{
this
.
serverSocket
=
new
ServerSocket
(
0
);
start
();
try
{
this
.
serverSocket
=
new
ServerSocket
(
0
);
start
();
if
(
LOG
.
isInfoEnabled
())
{
LOG
.
info
(
String
.
format
(
"Started BLOB server on port %d"
,
this
.
serverSocket
.
getLocalPort
()));
}
if
(
LOG
.
isInfoEnabled
())
{
LOG
.
info
(
String
.
format
(
"Started BLOB server on port %d"
,
this
.
serverSocket
.
getLocalPort
())
);
this
.
storageDir
=
BlobUtils
.
initStorageDirectory
();
}
catch
(
IOException
e
){
throw
new
IOException
(
"Could not create BlobServer with random port."
,
e
);
}
this
.
storageDir
=
BlobUtils
.
initStorageDirectory
();
}
/**
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
已删除
100644 → 0
浏览文件 @
c175ebe8
/*
* 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.execution
;
import
org.apache.flink.runtime.executiongraph.ExecutionAttemptID
;
import
org.apache.flink.runtime.jobgraph.JobID
;
import
org.apache.flink.runtime.jobgraph.JobVertexID
;
/**
* Implementing this interface allows classes to receive notifications about
* changes of a task's execution state.
*/
public
interface
ExecutionListener
{
void
executionStateChanged
(
JobID
jobID
,
JobVertexID
vertexId
,
int
subtask
,
ExecutionAttemptID
executionId
,
ExecutionState
newExecutionState
,
String
optionalMessage
);
}
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
浏览文件 @
8eadd3ec
...
...
@@ -38,7 +38,6 @@ import org.slf4j.LoggerFactory;
import
org.apache.flink.configuration.Configuration
;
import
org.apache.flink.runtime.JobException
;
import
org.apache.flink.runtime.blob.BlobKey
;
import
org.apache.flink.runtime.execution.ExecutionListener
;
import
org.apache.flink.runtime.execution.ExecutionState
;
import
org.apache.flink.runtime.instance.Instance
;
import
org.apache.flink.runtime.instance.InstanceConnectionInfo
;
...
...
@@ -97,10 +96,6 @@ public class ExecutionGraph {
private
final
List
<
BlobKey
>
requiredJarFiles
;
private
final
List
<
JobStatusListener
>
jobStatusListeners
;
private
final
List
<
ExecutionListener
>
executionListeners
;
private
final
List
<
ActorRef
>
jobStatusListenerActors
;
private
final
List
<
ActorRef
>
executionListenerActors
;
...
...
@@ -150,8 +145,6 @@ public class ExecutionGraph {
this
.
verticesInCreationOrder
=
new
ArrayList
<
ExecutionJobVertex
>();
this
.
currentExecutions
=
new
ConcurrentHashMap
<
ExecutionAttemptID
,
Execution
>();
this
.
jobStatusListeners
=
new
CopyOnWriteArrayList
<
JobStatusListener
>();
this
.
executionListeners
=
new
CopyOnWriteArrayList
<
ExecutionListener
>();
this
.
jobStatusListenerActors
=
new
CopyOnWriteArrayList
<
ActorRef
>();
this
.
executionListenerActors
=
new
CopyOnWriteArrayList
<
ActorRef
>();
...
...
@@ -638,14 +631,6 @@ public class ExecutionGraph {
// Listeners & Observers
// --------------------------------------------------------------------------------------------
public
void
registerJobStatusListener
(
JobStatusListener
jobStatusListener
)
{
this
.
jobStatusListeners
.
add
(
jobStatusListener
);
}
public
void
registerExecutionListener
(
ExecutionListener
executionListener
)
{
this
.
executionListeners
.
add
(
executionListener
);
}
public
void
registerJobStatusListener
(
ActorRef
listener
){
this
.
jobStatusListenerActors
.
add
(
listener
);
...
...
@@ -662,20 +647,6 @@ public class ExecutionGraph {
* @param error
*/
private
void
notifyJobStatusChange
(
JobStatus
newState
,
Throwable
error
)
{
if
(
jobStatusListeners
.
size
()
>
0
)
{
String
message
=
error
==
null
?
null
:
ExceptionUtils
.
stringifyException
(
error
);
for
(
JobStatusListener
listener
:
this
.
jobStatusListeners
)
{
try
{
listener
.
jobStatusHasChanged
(
this
,
newState
,
message
);
}
catch
(
Throwable
t
)
{
LOG
.
error
(
"Notification of job status change caused an error."
,
t
);
}
}
}
if
(
jobStatusListenerActors
.
size
()
>
0
){
String
message
=
error
==
null
?
null
:
ExceptionUtils
.
stringifyException
(
error
);
for
(
ActorRef
listener:
jobStatusListenerActors
){
...
...
@@ -696,17 +667,6 @@ public class ExecutionGraph {
*/
void
notifyExecutionChange
(
JobVertexID
vertexId
,
int
subtask
,
ExecutionAttemptID
executionID
,
ExecutionState
newExecutionState
,
Throwable
error
)
{
if
(
executionListeners
.
size
()
>
0
){
String
message
=
error
==
null
?
null
:
ExceptionUtils
.
stringifyException
(
error
);
for
(
ExecutionListener
listener
:
this
.
executionListeners
)
{
try
{
listener
.
executionStateChanged
(
jobID
,
vertexId
,
subtask
,
executionID
,
newExecutionState
,
message
);
}
catch
(
Throwable
t
){
LOG
.
error
(
"Notification of execution state change caused an error."
);
}
}
}
ExecutionJobVertex
vertex
=
getJobVertex
(
vertexId
);
if
(
executionListenerActors
.
size
()
>
0
){
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusListener.java
已删除
100644 → 0
浏览文件 @
c175ebe8
/*
* 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.executiongraph
;
import
org.apache.flink.runtime.jobgraph.JobStatus
;
/**
* This interface allows objects to receive notifications when the status of an observed job has changed.
*/
public
interface
JobStatusListener
{
/**
* Called when the status of the job changed.
*
* @param executionGraph The executionGraph representing the job.
* @param newJobStatus The new job status.
* @param optionalMessage An optional message (possibly <code>null</code>) that can be attached to the state change.
*/
void
jobStatusHasChanged
(
ExecutionGraph
executionGraph
,
JobStatus
newJobStatus
,
String
optionalMessage
);
}
flink-runtime/src/main/java/org/apache/flink/runtime/profiling/TaskManagerProfiler.java
浏览文件 @
8eadd3ec
...
...
@@ -29,23 +29,24 @@ import org.apache.flink.runtime.taskmanager.Task;
public
interface
TaskManagerProfiler
{
/**
* Registers a
n {@link org.apache.flink.runtime.execution.ExecutionListener
} object for profiling.
* Registers a
{@link org.apache.flink.runtime.taskmanager.Task
} object for profiling.
*
* @param task
* task to be register a profiling listener for
* @param jobConfiguration
* the job configuration sent with the task
*/
void
register
ExecutionListener
(
Task
task
,
Configuration
jobConfiguration
);
void
register
Task
(
Task
task
,
Configuration
jobConfiguration
);
/**
* Unregisters all previously register
{@link org.apache.flink.runtime.execution.ExecutionListener} objects for
* the vertex identified by the given ID.
* Unregisters all previously register
ed {@link org.apache.flink.runtime.taskmanager.Task}
*
objects for
the vertex identified by the given ID.
*
* @param id
* the ID of the vertex to unregister the {@link org.apache.flink.runtime.execution.ExecutionListener} objects for
* the ID of the vertex to unregister the
* {@link org.apache.flink.runtime.taskmanager.Task} objects for
*/
void
unregister
ExecutionListener
(
ExecutionAttemptID
id
);
void
unregister
Task
(
ExecutionAttemptID
id
);
/**
* Shuts done the task manager's profiling component
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
浏览文件 @
8eadd3ec
...
...
@@ -27,7 +27,6 @@ import org.slf4j.Logger;
import
org.slf4j.LoggerFactory
;
import
org.apache.flink.configuration.Configuration
;
import
org.apache.flink.runtime.execution.ExecutionListener
;
import
org.apache.flink.runtime.execution.ExecutionState
;
import
org.apache.flink.runtime.execution.RuntimeEnvironment
;
import
org.apache.flink.runtime.executiongraph.ExecutionAttemptID
;
...
...
@@ -62,9 +61,6 @@ public final class Task {
private
final
String
taskName
;
private
final
TaskManager
taskManager
;
private
final
List
<
ExecutionListener
>
executionListeners
=
new
CopyOnWriteArrayList
<
ExecutionListener
>();
private
final
List
<
ActorRef
>
executionListenerActors
=
new
CopyOnWriteArrayList
<
ActorRef
>();
...
...
@@ -354,7 +350,7 @@ public final class Task {
* the configuration attached to the job
*/
public
void
registerProfiler
(
TaskManagerProfiler
taskManagerProfiler
,
Configuration
jobConfiguration
)
{
taskManagerProfiler
.
register
ExecutionListener
(
this
,
jobConfiguration
);
taskManagerProfiler
.
register
Task
(
this
,
jobConfiguration
);
}
/**
...
...
@@ -365,7 +361,7 @@ public final class Task {
*/
public
void
unregisterProfiler
(
TaskManagerProfiler
taskManagerProfiler
)
{
if
(
taskManagerProfiler
!=
null
)
{
taskManagerProfiler
.
unregister
ExecutionListener
(
this
.
executionId
);
taskManagerProfiler
.
unregister
Task
(
this
.
executionId
);
}
}
...
...
@@ -373,24 +369,10 @@ public final class Task {
// State Listeners
// --------------------------------------------------------------------------------------------
public
void
registerExecutionListener
(
ExecutionListener
listener
)
{
if
(
listener
==
null
)
{
throw
new
IllegalArgumentException
();
}
this
.
executionListeners
.
add
(
listener
);
}
public
void
registerExecutionListener
(
ActorRef
listener
){
executionListenerActors
.
add
(
listener
);
}
public
void
unregisterExecutionListener
(
ExecutionListener
listener
)
{
if
(
listener
==
null
)
{
throw
new
IllegalArgumentException
();
}
this
.
executionListeners
.
remove
(
listener
);
}
public
void
unregisterExecutionListener
(
ActorRef
listener
){
executionListenerActors
.
remove
(
listener
);
}
...
...
@@ -400,15 +382,6 @@ public final class Task {
LOG
.
info
(
getTaskNameWithSubtasks
()
+
" switched to "
+
newState
+
(
message
==
null
?
""
:
" : "
+
message
));
}
for
(
ExecutionListener
listener
:
this
.
executionListeners
)
{
try
{
listener
.
executionStateChanged
(
jobId
,
vertexId
,
subtaskIndex
,
executionId
,
newState
,
message
);
}
catch
(
Throwable
t
)
{
LOG
.
error
(
"Error while calling execution listener."
,
t
);
}
}
for
(
ActorRef
listener:
executionListenerActors
){
listener
.
tell
(
new
ExecutionGraphMessages
.
ExecutionStateChanged
(
jobId
,
vertexId
,
taskName
,
numberOfSubtasks
,
subtaskIndex
,
...
...
flink-runtime/src/main/resources/web-docs-infoserver/js/jquery-1.10.2.js
已删除
100755 → 0
浏览文件 @
c175ebe8
此差异已折叠。
点击以展开。
flink-runtime/resources/web-docs-infoserver/js/jquery-2.1.0.js
→
flink-runtime/
src/main/
resources/web-docs-infoserver/js/jquery-2.1.0.js
浏览文件 @
8eadd3ec
文件已移动
flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
浏览文件 @
8eadd3ec
...
...
@@ -378,10 +378,6 @@ class TaskManager(val connectionInfo: InstanceConnectionInfo, val jobManagerAkka
val
receiver
=
this
.
self
val
taskName
=
runningTasks
(
executionID
).
getTaskName
val
numberOfSubtasks
=
runningTasks
(
executionID
).
getNumberOfSubtasks
val
indexOfSubtask
=
runningTasks
(
executionID
).
getSubtaskIndex
futureResponse
.
mapTo
[
Boolean
].
onComplete
{
case
Success
(
result
)
=>
if
(!
result
||
executionState
==
ExecutionState
.
FINISHED
||
executionState
==
...
...
@@ -390,7 +386,7 @@ class TaskManager(val connectionInfo: InstanceConnectionInfo, val jobManagerAkka
}
case
Failure
(
t
)
=>
log
.
error
(
t
,
s
"Execution state change notification failed for task ${executionID} "
+
s
"
($indexOfSubtask/$numberOfSubtasks)
of job ${jobID}."
)
s
"of job ${jobID}."
)
}
}
...
...
flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java
浏览文件 @
8eadd3ec
...
...
@@ -51,6 +51,7 @@ public class LocalExecutorITCase {
LocalExecutor
executor
=
new
LocalExecutor
();
executor
.
setDefaultOverwriteFiles
(
true
);
executor
.
setTaskManagerNumSlots
(
DOP
);
executor
.
setPrintStatusDuringExecution
(
false
);
executor
.
start
();
executor
.
executePlan
(
wc
.
getPlan
(
Integer
.
valueOf
(
DOP
).
toString
(),
inFile
.
toURI
().
toString
(),
...
...
flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
浏览文件 @
8eadd3ec
...
...
@@ -30,44 +30,46 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import
org.apache.flink.api.java.io.LocalCollectionOutputFormat
;
import
org.apache.flink.api.java.operators.DeltaIteration
;
import
org.apache.flink.api.java.tuple.Tuple2
;
import
org.apache.flink.test.util.JavaProgramTestBase
;
import
org.junit.Test
;
@SuppressWarnings
(
"serial"
)
public
class
DeltaIterationNotDependingOnSolutionSetITCase
{
public
class
DeltaIterationNotDependingOnSolutionSetITCase
extends
JavaProgramTestBase
{
private
final
List
<
Tuple2
<
Long
,
Long
>>
result
=
new
ArrayList
<
Tuple2
<
Long
,
Long
>>();
@
Test
p
ublic
void
testDeltaIterationNotDependingOnSolutionSet
()
{
@
Override
p
rotected
void
testProgram
()
throws
Exception
{
try
{
final
List
<
Tuple2
<
Long
,
Long
>>
result
=
new
ArrayList
<
Tuple2
<
Long
,
Long
>>();
ExecutionEnvironment
env
=
ExecutionEnvironment
.
getExecutionEnvironment
();
env
.
setDegreeOfParallelism
(
1
);
DataSet
<
Tuple2
<
Long
,
Long
>>
input
=
env
.
generateSequence
(
0
,
9
).
map
(
new
Duplicator
<
Long
>());
DeltaIteration
<
Tuple2
<
Long
,
Long
>,
Tuple2
<
Long
,
Long
>>
iteration
=
input
.
iterateDelta
(
input
,
5
,
1
);
iteration
.
closeWith
(
iteration
.
getWorkset
(),
iteration
.
getWorkset
().
map
(
new
TestMapper
()))
.
output
(
new
LocalCollectionOutputFormat
<
Tuple2
<
Long
,
Long
>>(
result
));
.
output
(
new
LocalCollectionOutputFormat
<
Tuple2
<
Long
,
Long
>>(
result
));
env
.
execute
();
boolean
[]
present
=
new
boolean
[
50
];
for
(
Tuple2
<
Long
,
Long
>
t
:
result
)
{
present
[
t
.
f0
.
intValue
()]
=
true
;
}
for
(
int
i
=
0
;
i
<
present
.
length
;
i
++)
{
assertTrue
(
String
.
format
(
"Missing tuple (%d, %d)"
,
i
,
i
),
present
[
i
]);
}
}
catch
(
Exception
e
)
{
e
.
printStackTrace
();
fail
(
e
.
getMessage
());
}
}
@Override
protected
void
postSubmit
()
{
boolean
[]
present
=
new
boolean
[
50
];
for
(
Tuple2
<
Long
,
Long
>
t
:
result
)
{
present
[
t
.
f0
.
intValue
()]
=
true
;
}
for
(
int
i
=
0
;
i
<
present
.
length
;
i
++)
{
assertTrue
(
String
.
format
(
"Missing tuple (%d, %d)"
,
i
,
i
),
present
[
i
]);
}
}
private
static
final
class
Duplicator
<
T
>
implements
MapFunction
<
T
,
Tuple2
<
T
,
T
>>
{
@Override
public
Tuple2
<
T
,
T
>
map
(
T
value
)
{
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录