flink git commit: [FLINK-3082] Fixed confusing error about an interface that no longer exists
Repository: flink Updated Branches: refs/heads/master 55fd5f32d -> d359a974a [FLINK-3082] Fixed confusing error about an interface that no longer exists This closes #1411 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d359a974 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d359a974 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d359a974 Branch: refs/heads/master Commit: d359a974ae475ab36fd68dec1f182dc01545dc19 Parents: 55fd5f3 Author: Niels BasjesAuthored: Thu Nov 26 15:21:04 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 26 17:45:51 2015 +0100 -- .../flink/streaming/api/operators/StreamSource.java | 12 ++-- 1 file changed, 6 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/d359a974/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java index 56426f6..e80654a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java @@ -95,14 +95,14 @@ public class StreamSource extends AbstractUdfStreamOperator
[2/7] flink git commit: [FLINK-3056] [web-dashboard] Represent bytes in more readable form.
http://git-wip-us.apache.org/repos/asf/flink/blob/80512229/flink-runtime-web/web-dashboard/web/js/index.js -- diff --git a/flink-runtime-web/web-dashboard/web/js/index.js b/flink-runtime-web/web-dashboard/web/js/index.js index 5947de0..271f82e 100644 --- a/flink-runtime-web/web-dashboard/web/js/index.js +++ b/flink-runtime-web/web-dashboard/web/js/index.js @@ -271,18 +271,29 @@ angular.module('flinkApp').filter("amDurationFormatExtended", ["angularMomentCon return ''; } }; -}).filter("bytes", function() { - return function(bytes, precision) { -var number, units; -if (isNaN(parseFloat(bytes)) || !isFinite(bytes)) { - return "-"; +}).filter("humanizeBytes", function() { + return function(bytes) { +var converter, units; +units = ["B", "KB", "MB", "GB", "TB", "PB", "EB"]; +converter = function(value, power) { + var base; + base = Math.pow(1024, power); + if (value < base) { +return (value / base).toFixed(2) + " " + units[power]; + } else if (value < base * 1000) { +return (value / base).toPrecision(3) + " " + units[power]; + } else { +return converter(value, power + 1); + } +}; +if (typeof bytes === "undefined" || bytes === null) { + return ""; } -if (typeof precision === "undefined") { - precision = 1; +if (bytes < 1000) { + return bytes + " B"; +} else { + return converter(bytes, 1); } -units = ["bytes", "kB", "MB", "GB", "TB", "PB"]; -number = Math.floor(Math.log(bytes) / Math.log(1024)); -return (bytes / Math.pow(1024, Math.floor(number))).toFixed(precision) + " " + units[number]; }; }); @@ -1262,4 +1273,4 @@ angular.module('flinkApp').service('TaskManagersService', ["$http", "flinkConfig return this; }]); -//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsImNvbW1vbi9zZXJ2aWNlcy5jb2ZmZWUiLCJjb21tb24vc2VydmljZXMuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvam9ibWFuYWdlci9qb2JtYW5hZ2VyLmN0cmwuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JtYW5hZ2VyL2pvYm1hbmFnZXIuc3ZjLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5jdHJsLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuZGlyLmNvZmZlZSIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5qcyIsIm1vZHVsZXMvam9icy9qb2JzLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuanMiLCJtb2R1bGVzL292ZXJ2aWV3L292ZXJ2aWV3LmN0cmwuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5 hZ2VyLmN0cmwuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5jdHJsLmpzIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5zdmMuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5zdmMuanMiXSwibmFtZXMiOltdLCJtYXBwaW5ncyI6IkFBa0JBLFFBQVEsT0FBTyxZQUFZLENBQUMsYUFBYSxrQkFJeEMsbUJBQUksU0FBQyxZQUFEO0VBQ0gsV0FBVyxpQkFBaUI7RUNyQjVCLE9Ec0JBLFdBQVcsY0FBYyxXQUFBO0lBQ3ZCLFdBQVcsaUJBQWlCLENBQUMsV0FBVztJQ3JCeEMsT0RzQkEsV0FBVyxlQUFlOztJQUk3QixNQUFNLGVBQWU7RUFDcEIsb0JBQW9CO0dBS3JCLCtEQUFJLFNBQUMsYUFBYSxhQUFhLGFBQWEsV0FBeEM7RUMzQkgsT0Q0QkEsWUFBWSxhQUFhLEtBQUssU0FBQyxRQUFEO0lBQzVCLFFBQVEsT0FBTyxhQUFhO0lBRTVCLFlBQVk7SUM1QlosT0Q4QkEsVUFBVSxXQUFBO01DN0JSLE9EOEJBLFlBQVk7T0FDWixZQUFZOztJQUtqQixpQ0FBTyxTQUFDLHVCQUFEO0VDaENOLE9EaUNBLHNCQUFzQjtJQUl2QixnREFBTyxTQUFDLGdCQUFnQixvQkFBakI7RUFDTixlQUFlLE1BQU0sWUFDbkI7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sZ0JBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sa0JBQ0w7SUFBQSxLQUFL O0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sY0FDTDtJQUFBLEtBQUs7SUFDTCxVQUFVO0lBQ1YsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sbUJBQ0w7SUFBQSxLQUFLO0lBQ0wsVUFBVTtJQUNWLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLDRCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxnQkFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxnQ0FDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsZ0JBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sdUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSw4QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsUUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxxQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLGVBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sa0JBQ
[6/7] flink git commit: [FLINK-2904] [web-dashboard] Fix truncation of task counts when the number is large.
[FLINK-2904] [web-dashboard] Fix truncation of task counts when the number is large. This closes #1321 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b80ecfdc Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b80ecfdc Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b80ecfdc Branch: refs/heads/master Commit: b80ecfdc211e1e91d6ebc5b79e257901933a047d Parents: cf91347 Author: Sachin GoelAuthored: Tue Nov 3 19:07:34 2015 +0530 Committer: Stephan Ewen Committed: Mon Nov 30 17:44:12 2015 +0100 -- flink-runtime-web/web-dashboard/app/styles/job.styl | 5 ++--- flink-runtime-web/web-dashboard/web/css/index.css | 5 ++--- 2 files changed, 4 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/b80ecfdc/flink-runtime-web/web-dashboard/app/styles/job.styl -- diff --git a/flink-runtime-web/web-dashboard/app/styles/job.styl b/flink-runtime-web/web-dashboard/app/styles/job.styl index 1051712..fa21d0e 100644 --- a/flink-runtime-web/web-dashboard/app/styles/job.styl +++ b/flink-runtime-web/web-dashboard/app/styles/job.styl @@ -34,9 +34,8 @@ .label-group .label display: inline-block -width: 2em -padding-left: 0.1em -padding-right: 0.1em +padding-left: 0.4em +padding-right: 0.4em margin: 0 border-right: 1px solid #ff border-radius(0) http://git-wip-us.apache.org/repos/asf/flink/blob/b80ecfdc/flink-runtime-web/web-dashboard/web/css/index.css -- diff --git a/flink-runtime-web/web-dashboard/web/css/index.css b/flink-runtime-web/web-dashboard/web/css/index.css index 45599ca..5a7c44f 100644 --- a/flink-runtime-web/web-dashboard/web/css/index.css +++ b/flink-runtime-web/web-dashboard/web/css/index.css @@ -445,9 +445,8 @@ livechart { } .label-group .label { display: inline-block; - width: 2em; - padding-left: 0.1em; - padding-right: 0.1em; + padding-left: 0.4em; + padding-right: 0.4em; margin: 0; border-right: 1px solid #fff; -webkit-border-radius: 0;
[7/7] flink git commit: [FLINK-2950] [ml] [docs] Fix markdown rendering problem in SVM documentation
[FLINK-2950] [ml] [docs] Fix markdown rendering problem in SVM documentation - Remove unnecessary indentation of table - Fix wrong `strong` end tag - Simplify lambda expression in map operation This closes #1312 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d5a6b13a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d5a6b13a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d5a6b13a Branch: refs/heads/master Commit: d5a6b13ab1ff6f42369b6b1cd2aad73bd6910362 Parents: 8051222 Author: Chiwan ParkAuthored: Sun Nov 1 11:15:26 2015 +0900 Committer: Stephan Ewen Committed: Mon Nov 30 17:44:13 2015 +0100 -- docs/libs/ml/svm.md | 202 +++ 1 file changed, 101 insertions(+), 101 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/d5a6b13a/docs/libs/ml/svm.md -- diff --git a/docs/libs/ml/svm.md b/docs/libs/ml/svm.md index 89f7e70..c344979 100644 --- a/docs/libs/ml/svm.md +++ b/docs/libs/ml/svm.md @@ -87,106 +87,106 @@ the algorithm's performance. The SVM implementation can be controlled by the following parameters: - - - -Parameters -Description - - - - - -Blocks - - -Sets the number of blocks into which the input data will be split. -On each block the local stochastic dual coordinate ascent method is executed. -This number should be set at least to the degree of parallelism. -If no value is specified, then the parallelism of the input DataSet is used as the number of blocks. -(Default value: None) - - - - -Iterations - - -Defines the maximum number of iterations of the outer loop method. -In other words, it defines how often the SDCA method is applied to the blocked data. -After each iteration, the locally computed weight vector updates have to be reduced to update the global weight vector value. -The new weight vector is broadcast to all SDCA tasks at the beginning of each iteration. -(Default value: 10) - - - - -LocalIterations - - -Defines the maximum number of SDCA iterations. -In other words, it defines how many data points are drawn from each local data block to calculate the stochastic dual coordinate ascent. -(Default value: 10) - - - - -Regularization - - -Defines the regularization constant of the SVM algorithm. -The higher the value, the smaller will the 2-norm of the weight vector be. -In case of a SVM with hinge loss this means that the SVM margin will be wider even though it might contain some false classifications. -(Default value: 1.0) - - - - -Stepsize - - -Defines the initial step size for the updates of the weight vector. -The larger the step size is, the larger will be the contribution of the weight vector updates to the next weight vector value. -The effective scaling of the updates is $\frac{stepsize}{blocks}$. -This value has to be tuned in case that the algorithm becomes unstable. -(Default value: 1.0) - - - - -ThresholdValue - - -Defines the limiting value for the decision function above which examples are labeled as -positive (+1.0). Examples with a decision function value below this value are classified -as negative (-1.0). In order to get the raw decision function values you need to indicate it by -using the OutputDecisionFunction parameter. (Default value: 0.0) - - - - -OutputDecisionFunction - - -Determines whether the predict and evaluate functions of the SVM should return the distance -to the separating hyperplane, or binary class labels. Setting this to true will -return the raw distance to the hyperplane for each example. Setting it to false will -return the binary class label (+1.0, -1.0) (Default value: false<\strong>) - - - - - Seed - - - Defines the seed to initialize the random number generator. - The seed directly controls which data points are chosen for the SDCA method. - (Default value: Random Long Integer) - -
[3/7] flink git commit: [FLINK-3056] [web-dashboard] Represent bytes in more readable form.
[FLINK-3056] [web-dashboard] Represent bytes in more readable form. Bytes are now displayed in the following fashion: 1. For [0, 1000) units, display three significant digits. 2. For [1000,1024) units, display 2 decimal points for the next higher unit. For example, 1010 KB is displayed as 0.99 MB, 10 MB is displayed as 10.0 MB and 230 MB is displayed as such. This closes #1419 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/80512229 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/80512229 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/80512229 Branch: refs/heads/master Commit: 80512229c487537e59688d35ffdfb10dc18a8ac9 Parents: 6bac921 Author: Sachin GoelAuthored: Sat Nov 28 16:54:23 2015 +0530 Committer: Stephan Ewen Committed: Mon Nov 30 17:44:12 2015 +0100 -- .../jobs/job.plan.node-list.overview.jade | 4 +-- .../partials/jobs/job.plan.node.subtasks.jade | 6 ++-- .../app/partials/taskmanager/index.jade | 6 ++-- .../taskmanager/taskmanager.metrics.jade| 33 +--- .../app/scripts/common/filters.coffee | 21 - flink-runtime-web/web-dashboard/web/js/index.js | 33 +--- .../jobs/job.plan.node-list.overview.html | 4 +-- .../partials/jobs/job.plan.node.subtasks.html | 4 +-- .../web/partials/taskmanager/index.html | 6 ++-- .../taskmanager/taskmanager.metrics.html| 24 +++--- 10 files changed, 85 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/80512229/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.overview.jade -- diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.overview.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.overview.jade index 54bd29c..ef9257d 100644 --- a/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.overview.jade +++ b/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.overview.jade @@ -39,9 +39,9 @@ table.table.table-body-hover.table-clickable.table-activable span(ng-if="v.duration > -1" title="{{v.duration | humanizeDuration:false}}") {{v.duration | humanizeDuration:true}} td.td-long {{ v.name | humanizeText }} - td {{ v.metrics['read-bytes'] | number }} + td(title="{{v.metrics['read-bytes']}} bytes") {{ v.metrics['read-bytes'] | humanizeBytes }} td {{ v.metrics['read-records'] | number }} - td {{ v.metrics['write-bytes'] | number }} + td(title="{{v.metrics['write-bytes']}} bytes") {{ v.metrics['write-bytes'] | humanizeBytes }} td {{ v.metrics['write-records'] | number }} td .label-group http://git-wip-us.apache.org/repos/asf/flink/blob/80512229/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node.subtasks.jade -- diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node.subtasks.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node.subtasks.jade index 6350325..259b364 100644 --- a/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node.subtasks.jade +++ b/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node.subtasks.jade @@ -43,11 +43,13 @@ table.table.table-hover.table-clickable.table-activable.table-inner(ng-if="subta span(ng-if="subtask.duration > -1" title="{{subtask.duration | humanizeDuration:false}}") {{subtask.duration | humanizeDuration:true}} td -span(ng-if="subtask.metrics['read-bytes'] > -1") {{ subtask.metrics['read-bytes'] | number }} +span(ng-if="subtask.metrics['read-bytes'] > -1" title="{{subtask.metrics['read-bytes']}} bytes") + | {{ subtask.metrics['read-bytes'] | humanizeBytes}} td span(ng-if="subtask.metrics['read-records'] > -1") {{ subtask.metrics['read-records'] | number }} td -span(ng-if="subtask.metrics['write-bytes'] > -1") {{ subtask.metrics['write-bytes'] | number }} +span(ng-if="subtask.metrics['write-bytes'] > -1" title="{{subtask.metrics['write-bytes']}} bytes") + | {{ subtask.metrics['write-bytes'] | humanizeBytes}} td span(ng-if="subtask.metrics['write-records'] > -1") {{ subtask.metrics['write-records'] | number }} http://git-wip-us.apache.org/repos/asf/flink/blob/80512229/flink-runtime-web/web-dashboard/app/partials/taskmanager/index.jade -- diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanager/index.jade
[4/7] flink git commit: [FLINK-2351] [core] Remove IOFormat ConfigBuilders
[FLINK-2351] [core] Remove IOFormat ConfigBuilders This closes #1420 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6bac9214 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6bac9214 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6bac9214 Branch: refs/heads/master Commit: 6bac921445d8fdecc4951bd4b9342bce0994c5ba Parents: b80ecfd Author: zentolAuthored: Sat Nov 28 14:36:06 2015 +0100 Committer: Stephan Ewen Committed: Mon Nov 30 17:44:12 2015 +0100 -- .../api/common/io/DelimitedInputFormat.java | 95 .../flink/api/common/io/FileInputFormat.java| 66 -- .../flink/api/common/io/FileOutputFormat.java | 50 --- .../operators/base/FileDataSourceBase.java | 28 -- 4 files changed, 239 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6bac9214/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java index 78c6705..cb32fc3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/DelimitedInputFormat.java @@ -24,7 +24,6 @@ import java.util.ArrayList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.io.statistics.BaseStatistics; -import org.apache.flink.api.common.operators.base.FileDataSourceBase; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; @@ -616,98 +615,4 @@ public abstract class DelimitedInputFormat extends FileInputFormat { * The configuration key to set the number of samples to take for the statistics. */ private static final String NUM_STATISTICS_SAMPLES = "delimited-format.numSamples"; - - // --- Config Builder - - - /** -* Creates a configuration builder that can be used to set the input format's parameters to the config in a fluent -* fashion. -* -* @return A config builder for setting parameters. -*/ - public static ConfigBuilder configureDelimitedFormat(FileDataSourceBase target) { - return new ConfigBuilder(target.getParameters()); - } - - /** -* Abstract builder used to set parameters to the input format's configuration in a fluent way. -*/ - protected static class AbstractConfigBuilder extends FileInputFormat.AbstractConfigBuilder { - - private static final String NEWLINE_DELIMITER = "\n"; - - // - - /** -* Creates a new builder for the given configuration. -* -* @param config The configuration into which the parameters will be written. -*/ - protected AbstractConfigBuilder(Configuration config) { - super(config); - } - - // - - /** -* Sets the delimiter to be a single character, namely the given one. The character must be within -* the value range 0 to 127. -* -* @param delimiter The delimiter character. -* @return The builder itself. -*/ - public T recordDelimiter(char delimiter) { - if (delimiter == '\n') { - this.config.setString(RECORD_DELIMITER, NEWLINE_DELIMITER); - } else { - this.config.setString(RECORD_DELIMITER, String.valueOf(delimiter)); - } - @SuppressWarnings("unchecked") - T ret = (T) this; - return ret; - } - - /** -* Sets the delimiter to be the given string. The string will be converted to bytes for more efficient -* comparison during input parsing. The conversion will be done using the platforms default charset. -* -* @param delimiter The delimiter string. -
[5/7] flink git commit: [FLINK-3083] [docs] Add docs on how to configure streaming fault tolerance.
[FLINK-3083] [docs] Add docs on how to configure streaming fault tolerance. This closes #1413 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/cf913476 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/cf913476 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/cf913476 Branch: refs/heads/master Commit: cf913476965051d2ca38f3e95a84246bb7de712e Parents: 2b358cd Author: Stephan EwenAuthored: Thu Nov 26 16:45:45 2015 +0100 Committer: Stephan Ewen Committed: Mon Nov 30 17:44:12 2015 +0100 -- docs/_includes/navbar.html | 3 +- docs/apis/fault_tolerance.md | 265 ++ docs/apis/streaming_guide.md | 131 +-- 3 files changed, 268 insertions(+), 131 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/cf913476/docs/_includes/navbar.html -- diff --git a/docs/_includes/navbar.html b/docs/_includes/navbar.html index 62bdce8..c565feb 100644 --- a/docs/_includes/navbar.html +++ b/docs/_includes/navbar.html @@ -81,8 +81,9 @@ under the License. Python API Beta -Interactive Scala Shell +Fault Tolerance State in Streaming Programs +Interactive Scala Shell DataSet Transformations Best Practices Connectors (DataSet API) http://git-wip-us.apache.org/repos/asf/flink/blob/cf913476/docs/apis/fault_tolerance.md -- diff --git a/docs/apis/fault_tolerance.md b/docs/apis/fault_tolerance.md new file mode 100644 index 000..677ff95 --- /dev/null +++ b/docs/apis/fault_tolerance.md @@ -0,0 +1,265 @@ +--- +title: "Fault Tolerance" +is_beta: false +--- + + + + +Flink's fault tolerance mechanism recovers programs in the presence of failures and +continues to execute them. Such failures include machine hardware failures, network failures, +transient program failures, etc. + +* This will be replaced by the TOC +{:toc} + + +Streaming Fault Tolerance (DataStream API) +-- + +Flink has a checkpointing mechanism that recovers streaming jobs after failues. The checkpointing mechanism requires a *persistent* (or *durable*) source that +can be asked for prior records again (Apache Kafka is a good example of such a source). + +The checkpointing mechanism stores the progress in the data sources and data sinks, the state of windows, as well as the user-defined state (see [Working with State]({{ site.baseurl }}/apis/streaming_guide.html#working-with-state)) consistently to provide *exactly once* processing semantics. Where the checkpoints are stored (e.g., JobManager memory, file system, database) depends on the configured [state backend]({{ site.baseurl }}/apis/state_backends.html). + +The [docs on streaming fault tolerance]({{ site.baseurl }}/internals/stream_checkpointing.html) describe in detail the technique behind Flink's streaming fault tolerance mechanism. + +To enable checkpointing, call `enableCheckpointing(n)` on the `StreamExecutionEnvironment`, where *n* is the checkpoint interval in milliseconds. + +Other parameters for checkpointing include: + +- *Number of retries*: The `setNumberOfExecutionRerties()` method defines how many times the job is restarted after a failure. + When checkpointing is activated, but this value is not explicitly set, the job is restarted infinitely often. + +- *exactly-once vs. at-least-once*: You can optionally pass a mode to the `enableCheckpointing(n)` method to choose between the two guarantee levels. + Exactly-once is preferrable for most applications. At-least-once may be relevant for certain super-low-latency (consistently few milliseconds) applications. + +- *number of concurrent checkpoints*: By default, the system will not trigger another checkpoint while one is still in progress. This ensures that the topology does not spend too much time on checkpoints and not make progress with processing the streams. It is possible to allow for multiple overlapping checkpoints, which is interesting for pipelines that have a certain processing delay (for example because the functions call external services that need some time to respond) but that still want to do very frequent checkpoints (100s of milliseconds) to re-process very little upon failures. + +- *checkpoint timeout*: The time after which a checkpoint-in-progress is aborted, if it did not complete until then. + + + +{% highlight java %} +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + +// start a checkpoint every 1000 ms
[1/7] flink git commit: [FLINK-3056] [web-dashboard] Represent bytes in more readable form.
Repository: flink Updated Branches: refs/heads/master 2b358cde0 -> d5a6b13ab http://git-wip-us.apache.org/repos/asf/flink/blob/80512229/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html -- diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html index daba67d..4d05cc3 100644 --- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html +++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html @@ -38,9 +38,9 @@ limitations under the License. {{ v['end-time'] | amDateFormat:'-MM-DD, H:mm:ss' }} {{v.duration | humanizeDuration:true}} {{ v.name | humanizeText }} - {{ v.metrics['read-bytes'] | number }} + {{ v.metrics['read-bytes'] | humanizeBytes }} {{ v.metrics['read-records'] | number }} - {{ v.metrics['write-bytes'] | number }} + {{ v.metrics['write-bytes'] | humanizeBytes }} {{ v.metrics['write-records'] | number }} http://git-wip-us.apache.org/repos/asf/flink/blob/80512229/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html -- diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html index 147b14d..874ed88 100644 --- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html +++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html @@ -38,9 +38,9 @@ limitations under the License. {{ subtask['start-time'] | amDateFormat:'-MM-DD, H:mm:ss' }} {{ subtask['end-time'] | amDateFormat:'-MM-DD, H:mm:ss' }} {{subtask.duration | humanizeDuration:true}} - {{ subtask.metrics['read-bytes'] | number }} + {{ subtask.metrics['read-bytes'] | humanizeBytes}} {{ subtask.metrics['read-records'] | number }} - {{ subtask.metrics['write-bytes'] | number }} + {{ subtask.metrics['write-bytes'] | humanizeBytes}} {{ subtask.metrics['write-records'] | number }} {{ subtask.attempt + 1 }} {{ subtask.host }} http://git-wip-us.apache.org/repos/asf/flink/blob/80512229/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html -- diff --git a/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html b/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html index bf37409..f56dcbe 100644 --- a/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html +++ b/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html @@ -48,9 +48,9 @@ limitations under the License. {{ manager.slotsNumber }} {{ manager.freeSlots }} {{ manager.cpuCores }} -{{ manager.physicalMemory | bytes:MB }} -{{ manager.freeMemory | bytes:MB }} -{{ manager.managedMemory | bytes:MB }} +{{ manager.physicalMemory | humanizeBytes }} +{{ manager.freeMemory | humanizeBytes }} +{{ manager.managedMemory | humanizeBytes }} http://git-wip-us.apache.org/repos/asf/flink/blob/80512229/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.metrics.html -- diff --git a/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.metrics.html b/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.metrics.html index 2a8b51e..691490e 100644 --- a/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.metrics.html +++ b/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.metrics.html @@ -30,21 +30,21 @@ limitations under the License. Heap - - - + {{metrics.metrics.gauges['memory.heap.committed'].value | humanizeBytes}} + {{metrics.metrics.gauges['memory.heap.init'].value | humanizeBytes}} + {{metrics.metrics.gauges['memory.heap.max'].value | humanizeBytes}} Non-Heap - - - + {{metrics.metrics.gauges['memory.non-heap.committed'].value | humanizeBytes}} + {{metrics.metrics.gauges['memory.non-heap.init'].value | humanizeBytes}} + {{metrics.metrics.gauges['memory.non-heap.max'].value | humanizeBytes}} Total - - - + {{metrics.metrics.gauges['memory.total.committed'].value | humanizeBytes}} + {{metrics.metrics.gauges['memory.total.init'].value | humanizeBytes}} + {{metrics.metrics.gauges['memory.total.max'].value | humanizeBytes}} @@ -66,9 +66,9 @@ limitations under the
flink git commit: [FLINK-3084] [streaming] FsStateBackend backs up very small state directly with the metadata.
Repository: flink Updated Branches: refs/heads/master cd899f3be -> d1ea365ef [FLINK-3084] [streaming] FsStateBackend backs up very small state directly with the metadata. This closes #1423 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d1ea365e Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d1ea365e Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d1ea365e Branch: refs/heads/master Commit: d1ea365ef1fa979d40532bf3f114fc03284164bc Parents: cd899f3 Author: Stephan EwenAuthored: Thu Nov 26 18:46:49 2015 +0100 Committer: Stephan Ewen Committed: Tue Dec 1 09:44:17 2015 +0100 -- .../flink/runtime/state/StreamStateHandle.java | 14 +- .../state/filesystem/AbstractFileState.java | 4 +- .../filesystem/FileSerializableStateHandle.java | 3 +- .../state/filesystem/FileStreamStateHandle.java | 7 + .../state/filesystem/FsStateBackend.java| 250 ++- .../state/filesystem/FsStateBackendFactory.java | 10 +- .../state/memory/ByteStreamStateHandle.java | 8 + .../state/memory/SerializedStateHandle.java | 39 ++- .../runtime/state/FileStateBackendTest.java | 45 ++-- .../FsCheckpointStateOutputStreamTest.java | 128 ++ .../flink/hdfstests/FileStateBackendTest.java | 32 +-- .../src/test/resources/log4j-test.properties| 31 +++ 12 files changed, 461 insertions(+), 110 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/d1ea365e/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamStateHandle.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamStateHandle.java index 32c601e..891243b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamStateHandle.java @@ -18,11 +18,19 @@ package org.apache.flink.runtime.state; -import org.apache.flink.runtime.state.StateHandle; - import java.io.InputStream; +import java.io.Serializable; /** * A state handle that produces an input stream when resolved. */ -public interface StreamStateHandle extends StateHandle {} +public interface StreamStateHandle extends StateHandle { + + /** +* Converts this stream state handle into a state handle that de-serializes +* the stream into an object using Java's serialization mechanism. +* +* @return The state handle that automatically de-serializes. +*/ +StateHandle toSerializableHandle(); +} http://git-wip-us.apache.org/repos/asf/flink/blob/d1ea365e/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileState.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileState.java index d64e2c4..e0a42b4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileState.java @@ -23,6 +23,8 @@ import org.apache.flink.core.fs.Path; import java.io.IOException; +import static java.util.Objects.requireNonNull; + /** * Base class for state that is stored in a file. */ @@ -42,7 +44,7 @@ public abstract class AbstractFileState implements java.io.Serializable { * @param filePath The path to the file that stores the state. */ protected AbstractFileState(Path filePath) { - this.filePath = filePath; + this.filePath = requireNonNull(filePath); } /** http://git-wip-us.apache.org/repos/asf/flink/blob/d1ea365e/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java index 63336d1..edbbe69 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java @@ -24,13 +24,14 @@ import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.util.InstantiationUtil; import
[2/2] flink git commit: [FLINK-3051] [streaming] Add mechanisms to control the maximum number of concurrent checkpoints
[FLINK-3051] [streaming] Add mechanisms to control the maximum number of concurrent checkpoints This closes #1408 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/55fd5f32 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/55fd5f32 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/55fd5f32 Branch: refs/heads/master Commit: 55fd5f32d7ef0292a01192ab08456fae49b91791 Parents: 4097666 Author: Stephan EwenAuthored: Thu Nov 19 19:05:47 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 26 17:16:29 2015 +0100 -- .../checkpoint/CheckpointCoordinator.java | 402 --- .../CheckpointCoordinatorDeActivator.java | 11 +- .../runtime/checkpoint/PendingCheckpoint.java | 10 +- .../runtime/executiongraph/ExecutionGraph.java | 22 +- .../jobgraph/tasks/JobSnapshottingSettings.java | 55 ++- .../flink/runtime/jobmanager/JobManager.scala | 2 + .../checkpoint/CheckpointCoordinatorTest.java | 357 +++- .../checkpoint/CheckpointStateRestoreTest.java | 6 +- .../checkpoint/CoordinatorShutdownTest.java | 6 +- .../api/environment/CheckpointConfig.java | 221 ++ .../environment/StreamExecutionEnvironment.java | 61 +-- .../flink/streaming/api/graph/StreamGraph.java | 105 ++--- .../api/graph/StreamGraphGenerator.java | 11 - .../api/graph/StreamingJobGraphGenerator.java | 36 +- .../api/scala/StreamExecutionEnvironment.scala | 7 + 15 files changed, 990 insertions(+), 322 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/55fd5f32/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 09dd2d9..454b88a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -21,6 +21,7 @@ package org.apache.flink.runtime.checkpoint; import akka.actor.ActorSystem; import akka.actor.PoisonPill; import akka.actor.Props; + import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; @@ -34,6 +35,7 @@ import org.apache.flink.runtime.jobmanager.RecoveryMode; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete; import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +48,6 @@ import java.util.Map; import java.util.Timer; import java.util.TimerTask; import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -63,11 +64,12 @@ import static com.google.common.base.Preconditions.checkNotNull; */ public class CheckpointCoordinator { - private static final Logger LOG = LoggerFactory.getLogger(CheckpointCoordinator.class); + static final Logger LOG = LoggerFactory.getLogger(CheckpointCoordinator.class); /** The number of recent checkpoints whose IDs are remembered */ private static final int NUM_GHOST_CHECKPOINT_IDS = 16; + /** Coordinator-wide lock to safeguard the checkpoint updates */ private final Object lock = new Object(); @@ -83,35 +85,58 @@ public class CheckpointCoordinator { /** Tasks who need to be sent a message when a checkpoint is confirmed */ private final ExecutionVertex[] tasksToCommitTo; + /** Map from checkpoint ID to the pending checkpoint */ private final Map pendingCheckpoints; - /** -* Completed checkpoints. Implementations can be blocking. Make sure calls to methods -* accessing this don't block the job manager actor and run asynchronously. -*/ + /** Completed checkpoints. Implementations can be blocking. Make sure calls to methods +* accessing this don't block the job manager actor and run asynchronously. */ private final CompletedCheckpointStore completedCheckpointStore; + /** A list of recent checkpoint IDs, to identify late messages (vs invalid ones) */ private final ArrayDeque recentPendingCheckpoints; - /** -* Checkpoint ID
[1/2] flink git commit: [hotfix] Java-7-ify the ExecutionConfig class
Repository: flink Updated Branches: refs/heads/master dcce6def9 -> 55fd5f32d [hotfix] Java-7-ify the ExecutionConfig class Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4097666e Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4097666e Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4097666e Branch: refs/heads/master Commit: 4097666ef3bf595e20fef206e07aa0251fe7eb35 Parents: dcce6de Author: Stephan EwenAuthored: Thu Nov 19 18:03:55 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 26 10:51:43 2015 +0100 -- .../org/apache/flink/api/common/ExecutionConfig.java | 14 +++--- 1 file changed, 7 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/4097666e/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index b031441..a0d3363 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -88,7 +88,7 @@ public class ExecutionConfig implements Serializable { /** If set to true, progress updates are printed to System.out during execution */ private boolean printProgressDuringExecution = true; - private GlobalJobParameters globalJobParameters = null; + private GlobalJobParameters globalJobParameters; private long autoWatermarkInterval = 0; @@ -99,17 +99,17 @@ public class ExecutionConfig implements Serializable { // Serializers and types registered with Kryo and the PojoSerializer // we store them in linked maps/sets to ensure they are registered in order in all kryo instances. - private final LinkedHashMap registeredTypesWithKryoSerializers = new LinkedHashMap (); + private final LinkedHashMap registeredTypesWithKryoSerializers = new LinkedHashMap<>(); - private final LinkedHashMap > registeredTypesWithKryoSerializerClasses = new LinkedHashMap >(); + private final LinkedHashMap > registeredTypesWithKryoSerializerClasses = new LinkedHashMap<>(); - private final LinkedHashMap defaultKryoSerializers = new LinkedHashMap (); + private final LinkedHashMap defaultKryoSerializers = new LinkedHashMap<>(); - private final LinkedHashMap > defaultKryoSerializerClasses = new LinkedHashMap >(); + private final LinkedHashMap > defaultKryoSerializerClasses = new LinkedHashMap<>(); - private final LinkedHashSet registeredKryoTypes = new LinkedHashSet (); + private final LinkedHashSet registeredKryoTypes = new LinkedHashSet<>(); - private final LinkedHashSet registeredPojoTypes = new LinkedHashSet (); + private final LinkedHashSet registeredPojoTypes = new LinkedHashSet<>(); //
[2/2] flink git commit: [hotfix] [streaming] Fix instantiation of state backends from state backend factory.
[hotfix] [streaming] Fix instantiation of state backends from state backend factory. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3c5f363d Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3c5f363d Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3c5f363d Branch: refs/heads/master Commit: 3c5f363d816a1a529832f384fb504d4b6804a00b Parents: 8325bc6 Author: Stephan EwenAuthored: Thu Nov 19 00:24:56 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 19 10:23:30 2015 +0100 -- .../java/org/apache/flink/streaming/runtime/tasks/StreamTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/3c5f363d/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index bae0128..80c63da 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -525,7 +525,7 @@ public abstract class StreamTask Class clazz = Class.forName(backendName, false, userClassLoader).asSubclass(StateBackendFactory.class); - return (StateBackend) clazz.newInstance(); + return clazz.newInstance().createFromConfig(flinkConfig); } catch (ClassNotFoundException e) { throw new IllegalConfigurationException("Cannot find configured state backend: " + backendName); } catch (ClassCastException e) {
[1/2] flink git commit: [FLINK-3040] [docs] Add docs for State Backends
Repository: flink Updated Branches: refs/heads/master 8325bc6a6 -> 8b086eb91 [FLINK-3040] [docs] Add docs for State Backends Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8b086eb9 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8b086eb9 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8b086eb9 Branch: refs/heads/master Commit: 8b086eb918c05f9ea8a2cc3840ca538788b39b01 Parents: 3c5f363 Author: Stephan EwenAuthored: Thu Nov 19 00:31:48 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 19 10:23:30 2015 +0100 -- docs/_includes/navbar.html | 2 +- docs/apis/state_backends.md | 121 +++ 2 files changed, 122 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/8b086eb9/docs/_includes/navbar.html -- diff --git a/docs/_includes/navbar.html b/docs/_includes/navbar.html index 278f5ab..62bdce8 100644 --- a/docs/_includes/navbar.html +++ b/docs/_includes/navbar.html @@ -82,10 +82,10 @@ under the License. Interactive Scala Shell +State in Streaming Programs DataSet Transformations Best Practices Connectors (DataSet API) - Examples Local Execution Cluster Execution http://git-wip-us.apache.org/repos/asf/flink/blob/8b086eb9/docs/apis/state_backends.md -- diff --git a/docs/apis/state_backends.md b/docs/apis/state_backends.md new file mode 100644 index 000..ad191f9 --- /dev/null +++ b/docs/apis/state_backends.md @@ -0,0 +1,121 @@ +--- +title: "State Backends" +--- + + +Programs written in the [Data Stream API]({{ site.baseurl }}/apis/streaming_guide.html) often hold state in various forms: + +- Windows gather elements or aggregates until they are triggered +- Transformation functions may use the key/value state interface to store values +- Transformation functions may implement the `Checkpointed` interface to make their local variables fault tolerant + +See also [Working with State]({{ site.baseurl }}/apis/streaming_guide.html#working_with_state) in the streaming API guide. + +When checkpointing is activated, such state is persisted upon checkpoints to guard against data loss and recover consistently. +How the state is represented internally, and how and where it is persisted upon checkpoints depends on the +chosen **State Backend**. + + +## Available State Backends + +Out of the box, Flink bundles two state backends: *MemoryStateBacked* and *FsStateBackend*. If nothing else is configured, +the system will use the MemoryStateBacked. + + +### The MemoryStateBackend + +The *MemoryStateBacked* holds data internally as objects on the Java heap. Key/value state and window operators hold hash tables +that store the values, triggers, etc. + +Upon checkpoints, this state backend will snapshot the state and send it as part of the checkpoint acknowledgement messages to the +JobManager (master), which stores it on its heap as well. + +Limitations of the MemoryStateBackend: + + - The size of each individual state is by default limited to 5 MB. This value can be increased in the constructor of the MemoryStateBackend. + - Irrespective of the configured maximal state size, the state cannot be larger than the akka frame size (see [Configuration]({{ site.baseurl }}/setup/config.html)). + - The aggregate state must fit into the JobManager memory. + +The MemoryStateBackend is encouraged for: + + - Local development and debugging + - Jobs that do hold little state, such as jobs that consist only of record-at-a-time functions (Map, FlatMap, Filter, ...). The Kafka Consumer requires very little state. + + +### The FsStateBackend + +The *FsStateBackend* (FileSystemStateBackend) is configured with a file system URL (type, address, path), such as for example "hdfs://namenode:40010/flink/checkpoints" or "file:///data/flink/checkpoints". + +The FsStateBackend holds in-flight data in the TaskManager's memory. Upon checkpoints, it writes state snapshots into files in the configured file system and directory. Minimal metadata is stored in the JobManager's memory (or, in high-availability mode, in the metadata checkpoint). + +The FsStateBackend is encouraged for: + + - Jobs with large state, long windows, large key/value states. + - All high-availability setups. + + +## Configuring a State Backend + +State backends can be configured per job. In addition, you can define a default state backend to be used when the +job does not explicitly define a state backend. + + +###
[1/2] flink git commit: [FLINK-3040] [docs] Add docs for State Backends
Repository: flink Updated Branches: refs/heads/release-0.10 a7e799b9f -> db456a761 [FLINK-3040] [docs] Add docs for State Backends Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/db456a76 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/db456a76 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/db456a76 Branch: refs/heads/release-0.10 Commit: db456a761480679f54136743237999049cb7476b Parents: ef70909 Author: Stephan EwenAuthored: Thu Nov 19 00:31:48 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 19 10:27:46 2015 +0100 -- docs/_includes/navbar.html | 2 +- docs/apis/state_backends.md | 121 +++ 2 files changed, 122 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/db456a76/docs/_includes/navbar.html -- diff --git a/docs/_includes/navbar.html b/docs/_includes/navbar.html index d39b330..30c0af0 100644 --- a/docs/_includes/navbar.html +++ b/docs/_includes/navbar.html @@ -82,10 +82,10 @@ under the License. Interactive Scala Shell +State in Streaming Programs DataSet Transformations Best Practices Connectors (DataSet API) - Examples Local Execution Cluster Execution http://git-wip-us.apache.org/repos/asf/flink/blob/db456a76/docs/apis/state_backends.md -- diff --git a/docs/apis/state_backends.md b/docs/apis/state_backends.md new file mode 100644 index 000..ad191f9 --- /dev/null +++ b/docs/apis/state_backends.md @@ -0,0 +1,121 @@ +--- +title: "State Backends" +--- + + +Programs written in the [Data Stream API]({{ site.baseurl }}/apis/streaming_guide.html) often hold state in various forms: + +- Windows gather elements or aggregates until they are triggered +- Transformation functions may use the key/value state interface to store values +- Transformation functions may implement the `Checkpointed` interface to make their local variables fault tolerant + +See also [Working with State]({{ site.baseurl }}/apis/streaming_guide.html#working_with_state) in the streaming API guide. + +When checkpointing is activated, such state is persisted upon checkpoints to guard against data loss and recover consistently. +How the state is represented internally, and how and where it is persisted upon checkpoints depends on the +chosen **State Backend**. + + +## Available State Backends + +Out of the box, Flink bundles two state backends: *MemoryStateBacked* and *FsStateBackend*. If nothing else is configured, +the system will use the MemoryStateBacked. + + +### The MemoryStateBackend + +The *MemoryStateBacked* holds data internally as objects on the Java heap. Key/value state and window operators hold hash tables +that store the values, triggers, etc. + +Upon checkpoints, this state backend will snapshot the state and send it as part of the checkpoint acknowledgement messages to the +JobManager (master), which stores it on its heap as well. + +Limitations of the MemoryStateBackend: + + - The size of each individual state is by default limited to 5 MB. This value can be increased in the constructor of the MemoryStateBackend. + - Irrespective of the configured maximal state size, the state cannot be larger than the akka frame size (see [Configuration]({{ site.baseurl }}/setup/config.html)). + - The aggregate state must fit into the JobManager memory. + +The MemoryStateBackend is encouraged for: + + - Local development and debugging + - Jobs that do hold little state, such as jobs that consist only of record-at-a-time functions (Map, FlatMap, Filter, ...). The Kafka Consumer requires very little state. + + +### The FsStateBackend + +The *FsStateBackend* (FileSystemStateBackend) is configured with a file system URL (type, address, path), such as for example "hdfs://namenode:40010/flink/checkpoints" or "file:///data/flink/checkpoints". + +The FsStateBackend holds in-flight data in the TaskManager's memory. Upon checkpoints, it writes state snapshots into files in the configured file system and directory. Minimal metadata is stored in the JobManager's memory (or, in high-availability mode, in the metadata checkpoint). + +The FsStateBackend is encouraged for: + + - Jobs with large state, long windows, large key/value states. + - All high-availability setups. + + +## Configuring a State Backend + +State backends can be configured per job. In addition, you can define a default state backend to be used when the +job does not explicitly define a state backend. +
[2/2] flink git commit: [hotfix] [streaming] Fix instantiation of state backends from state backend factory.
[hotfix] [streaming] Fix instantiation of state backends from state backend factory. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ef709093 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ef709093 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ef709093 Branch: refs/heads/release-0.10 Commit: ef709093b38ae12ea25b381aa54995d676aa3d33 Parents: a7e799b Author: Stephan EwenAuthored: Thu Nov 19 00:24:56 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 19 10:27:46 2015 +0100 -- .../java/org/apache/flink/streaming/runtime/tasks/StreamTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/ef709093/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index bae0128..80c63da 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -525,7 +525,7 @@ public abstract class StreamTask Class clazz = Class.forName(backendName, false, userClassLoader).asSubclass(StateBackendFactory.class); - return (StateBackend) clazz.newInstance(); + return clazz.newInstance().createFromConfig(flinkConfig); } catch (ClassNotFoundException e) { throw new IllegalConfigurationException("Cannot find configured state backend: " + backendName); } catch (ClassCastException e) {
flink git commit: [hotfix] [docs] Fix errors in streaming docs for fold()
Repository: flink Updated Branches: refs/heads/master 38cf0c610 -> 2f013a204 [hotfix] [docs] Fix errors in streaming docs for fold() Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2f013a20 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2f013a20 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2f013a20 Branch: refs/heads/master Commit: 2f013a20414a825bb6fc31dfc96ff562141e209b Parents: 38cf0c6 Author: Stephan EwenAuthored: Thu Nov 19 20:33:17 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 19 20:33:17 2015 +0100 -- docs/apis/streaming_guide.md | 48 +++ 1 file changed, 28 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/2f013a20/docs/apis/streaming_guide.md -- diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index fb6d86a..db1f3a7 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -519,22 +519,23 @@ keyedStream.reduce(new ReduceFunction() { - FoldDataStream DataStream + FoldKeyedStream DataStream A "rolling" fold on a keyed data stream with an initial value. Combines the current element with the last folded value and emits the new value. - A fold function that creates a stream of partial sums: + A fold function that, when applied on the sequence (1,2,3,4,5), + emits the sequence "start-1", "start-1-2", "start-1-2-3", ... {% highlight java %} -keyedStream.fold(0, new ReduceFunction() { - @Override - public Integer fold(Integer accumulator, Integer value) - throws Exception { - return accumulator + value; - } -}); +DataStream result = + keyedStream.fold("start", new FoldFunction () { +@Override +public String fold(String current, Integer value) { +return current + "-" + value; +} + }); {% endhighlight %} @@ -621,11 +622,13 @@ windowedStream.reduce (new ReduceFunction () { Window FoldWindowedStream DataStream -Applies a functional fold function to the window and returns the folded value. +Applies a functional fold function to the window and returns the folded value. + The example function, when applied on the sequence (1,2,3,4,5), + folds the sequence into the string "start-1-2-3-4-5": {% highlight java %} -windowedStream.fold (new Tuple2 ("Sum of all", 0), new FoldFunction () { -public Tuple2 fold(Tuple2 acc, Tuple2 value) throws Exception { -return new Tuple2 (acc.f0, acc.f1 + value.f1); +windowedStream.fold("start-", new FoldFunction () { +public String fold(String current, Integer value) { +return current + "-" + value; } }; {% endhighlight %} @@ -884,16 +887,18 @@ keyedStream.reduce { _ + _ } - FoldDataStream DataStream + FoldKeyedStream DataStream A "rolling" fold on a keyed data stream with an initial value. Combines the current element with the last folded value and emits the new value. - A fold function that creates a stream of partial sums: + A fold function that, when applied on the sequence (1,2,3,4,5), + emits the sequence "start-1", "start-1-2", "start-1-2-3", ... {% highlight scala %} -keyedStream.fold { 0, _ + _ } +val result: DataStream[String] = +keyedStream.fold("start", (str, i) => { str + "-" + i }) {% endhighlight %} @@ -965,10 +970,13 @@ windowedStream.reduce { _ + _ } Window FoldWindowedStream DataStream -Applies a functional fold function to the window and returns the folded value. -{% highlight java %} -windowedStream.fold { 0, _ + _ } -{% endhighlight %} +Applies a functional fold function to the window and returns the folded value. + The example function, when applied on the sequence (1,2,3,4,5), + folds the sequence into the string "start-1-2-3-4-5": + {% highlight scala %} +val result: DataStream[String] = +windowedStream.fold("start", (str, i) => { str + "-" + i }) + {% endhighlight %}
[2/2] flink git commit: [FLINK-3048] [tests] Increase stability of DataSinkTaskTest
[FLINK-3048] [tests] Increase stability of DataSinkTaskTest Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/93622001 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/93622001 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/93622001 Branch: refs/heads/master Commit: 93622001e499fa04bb5c4a63b1b3ed09b270f5b9 Parents: ff52d28 Author: Stephan EwenAuthored: Thu Nov 19 15:50:46 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 19 15:50:46 2015 +0100 -- .../flink/runtime/operators/DataSinkTask.java | 2 +- .../runtime/operators/DataSinkTaskTest.java | 38 2 files changed, 25 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/93622001/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java index d20bb89..addceea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java @@ -257,7 +257,7 @@ public class DataSinkTask extends AbstractInvokable { } } - BatchTask.clearReaders(new MutableReader[]{inputReader}); + BatchTask.clearReaders(new MutableReader[]{inputReader}); } if (!this.taskCanceled) { http://git-wip-us.apache.org/repos/asf/flink/blob/93622001/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java index b741b64..6221706 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java @@ -31,12 +31,14 @@ import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.types.IntValue; import org.apache.flink.types.Record; + import org.junit.After; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,10 +51,13 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Set; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + @RunWith(PowerMockRunner.class) @PrepareForTest({Task.class, ResultPartitionWriter.class}) -public class DataSinkTaskTest extends TaskTestBase -{ +public class DataSinkTaskTest extends TaskTestBase { + private static final Logger LOG = LoggerFactory.getLogger(DataSinkTaskTest.class); private static final int MEMORY_MANAGER_SIZE = 3 * 1024 * 1024; @@ -358,8 +363,7 @@ public class DataSinkTaskTest extends TaskTestBase } @Test - public void testCancelDataSinkTask() { - + public void testCancelDataSinkTask() throws Exception { super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE); super.addInput(new InfiniteInputIterator(), 0); @@ -382,19 +386,25 @@ public class DataSinkTaskTest extends TaskTestBase }; taskRunner.start(); - TaskCancelThread tct = new TaskCancelThread(1, taskRunner, testTask); - tct.start(); - - try { - tct.join(); - taskRunner.join(); - } catch(InterruptedException ie) { - Assert.fail("Joining threads failed"); + File tempTestFile = new File(this.tempTestPath); + + // wait until the task created the file + long deadline = System.currentTimeMillis() + 6; + while (!tempTestFile.exists() && System.currentTimeMillis() < deadline) { + Thread.sleep(10); } + assertTrue("Task did not create file within 60 seconds", tempTestFile.exists()); + + // cancel the task + Thread.sleep(500); + testTask.cancel(); + taskRunner.interrupt();
flink git commit: [FLINK-2429] [streaming] Deprecate the "enableCheckpointing()" method with no interval argument.
Repository: flink Updated Branches: refs/heads/master 1600fe2c2 -> 68a69c76a [FLINK-2429] [streaming] Deprecate the "enableCheckpointing()" method with no interval argument. This closes #1382 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/68a69c76 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/68a69c76 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/68a69c76 Branch: refs/heads/master Commit: 68a69c76ae66908658bed17a95dda7b74bc72675 Parents: 1600fe2 Author: Stephan EwenAuthored: Tue Nov 17 14:12:45 2015 +0100 Committer: Stephan Ewen Committed: Mon Nov 23 10:04:23 2015 +0100 -- .../streaming/api/environment/StreamExecutionEnvironment.java | 3 +++ 1 file changed, 3 insertions(+) -- http://git-wip-us.apache.org/repos/asf/flink/blob/68a69c76/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 5cc0007..72722bf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -332,7 +332,10 @@ public abstract class StreamExecutionEnvironment { * the moment. For that reason, iterative jobs will not be started if used * with enabled checkpointing. To override this mechanism, use the * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method. +* +* @deprecated Use {@link #enableCheckpointing(long)} instead. */ + @Deprecated public StreamExecutionEnvironment enableCheckpointing() { enableCheckpointing(500, CheckpointingMode.EXACTLY_ONCE); return this;
[2/2] flink git commit: [FLINK-3005] [core] Bump commons-collections version to fix object deserialization remote command execution vulnerability
[FLINK-3005] [core] Bump commons-collections version to fix object deserialization remote command execution vulnerability This closes #1381 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/7441799e Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7441799e Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7441799e Branch: refs/heads/release-0.10 Commit: 7441799ef7e526d352e3e03b96aa2be2ba30171b Parents: e977d82 Author: tedyuAuthored: Wed Nov 18 13:56:31 2015 -0800 Committer: Stephan Ewen Committed: Thu Nov 19 14:49:00 2015 +0100 -- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/7441799e/pom.xml -- diff --git a/pom.xml b/pom.xml index e161614..ac7eaba 100644 --- a/pom.xml +++ b/pom.xml @@ -210,7 +210,7 @@ under the License. commons-collections commons-collections - 3.2.1 + 3.2.2
[1/2] flink git commit: [FLINK-3043] [docs] Fix description of Kafka Consumer and Producer.
Repository: flink Updated Branches: refs/heads/master 6b253d9f8 -> 864357bac [FLINK-3043] [docs] Fix description of Kafka Consumer and Producer. This also adds to the deprecated classes pointers forward to the designated classes. This closes #1380 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/864357ba Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/864357ba Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/864357ba Branch: refs/heads/master Commit: 864357bacee3531d21a02c951c4b924fb0494eb6 Parents: 2061206 Author: Stephan EwenAuthored: Wed Nov 18 20:30:05 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 19 14:45:44 2015 +0100 -- docs/apis/kafka.md | 63 -- docs/apis/streaming_guide.md| 121 ++- .../connectors/kafka/FlinkKafkaProducer.java| 5 +- .../connectors/kafka/api/KafkaSink.java | 2 + .../api/persistent/PersistentKafkaSource.java | 5 + 5 files changed, 47 insertions(+), 149 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/864357ba/docs/apis/kafka.md -- diff --git a/docs/apis/kafka.md b/docs/apis/kafka.md deleted file mode 100644 index 0c0790a..000 --- a/docs/apis/kafka.md +++ /dev/null @@ -1,63 +0,0 @@ -title: "Reading from Kafka" -is_beta: true - - - - - -Interact with [Apache Kafka](https://kafka.apache.org/) streams from Flink's APIs. - -* This will be replaced by the TOC -{:toc} - - -Kafka Connector - -### Background - -Flink provides special Kafka Connectors for reading and writing data to Kafka topics. -The Flink Kafka Consumer integrates with Flink's checkpointing mechanisms to provide different -processing guarantees (most importantly exactly-once guarantees). - -For exactly-once processing Flink can not rely on the auto-commit capabilities of the Kafka consumers. -The Kafka consumer might commit offsets to Kafka which have not been processed successfully. - -Flink provides different connector implementations for different use-cases and environments. - - - - -### How to read data from Kafka - - Choose appropriate package and class - -Please pick a package (maven artifact id) and class name for your use-case and environment. For most users, the `flink-connector-kafka-083` package and the `FlinkKafkaConsumer082` class are appropriate. - -| Package | Supported Since | Class | Kafka Version | Allows exactly once processing | Notes | -| - |-| -| -- | -- | -| flink-connector-kafka | 0.9, 0.10 | `KafkaSource` | 0.8.1, 0.8.2 | **No**, does not participate in checkpointing at all. | Uses the old, high level KafkaConsumer API, autocommits to ZK by Kafka | -| flink-connector-kafka | 0.9, 0.10 | `PersistentKafkaSource` | 0.8.1, 0.8.2 | **No**, does not guarantee exactly-once processing, element order or strict partition assignment | Uses the old, high level KafkaConsumer API, offsets are committed into ZK manually | -| flink-connector-kafka-083 | 0.9.1 0.10 | `FlinkKafkaConsumer081` | 0.8.1 | **yes** | Uses the [SimpleConsumer](https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example) API of Kafka internally. Offsets are committed to ZK manually | -| flink-connector-kafka-083 | 0.9.1 0.10 | `FlinkKafkaConsumer082` | 0.8.2 | **yes** | Uses the [SimpleConsumer](https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example) API of Kafka internally. Offsets are committed to ZK manually | - - http://git-wip-us.apache.org/repos/asf/flink/blob/864357ba/docs/apis/streaming_guide.md -- diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index 1c97dd9..fb6d86a 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -3367,74 +3367,43 @@ with connectors. This connector provides access to event streams served by [Apache Kafka](https://kafka.apache.org/). -Flink provides special Kafka Connectors for reading and writing data to Kafka topics. -The Flink Kafka Consumer integrates with Flink's checkpointing mechanisms to provide different -processing guarantees (most importantly exactly-once guarantees). - -For exactly-once processing Flink can not rely on the auto-commit capabilities of the Kafka consumers. -The Kafka consumer might commit offsets to Kafka which have not been processed successfully. +Flink provides special Kafka Connectors for reading and writing data from/to Kafka topics. +The Flink Kafka Consumer integrates with Flink's checkpointing mechanism to
[2/2] flink git commit: [FLINK-3005] [core] Bump commons-collections version to fix object deserialization remote command execution vulnerability
[FLINK-3005] [core] Bump commons-collections version to fix object deserialization remote command execution vulnerability This closes #1381 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/20612063 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/20612063 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/20612063 Branch: refs/heads/master Commit: 206120631d97898c9396d74b2450eb36af17e06a Parents: 6b253d9 Author: tedyuAuthored: Wed Nov 18 13:56:31 2015 -0800 Committer: Stephan Ewen Committed: Thu Nov 19 14:45:44 2015 +0100 -- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/20612063/pom.xml -- diff --git a/pom.xml b/pom.xml index af4ff8d..95c7b6e 100644 --- a/pom.xml +++ b/pom.xml @@ -210,7 +210,7 @@ under the License. commons-collections commons-collections - 3.2.1 + 3.2.2
[1/2] flink git commit: [FLINK-3043] [docs] Fix description of Kafka Consumer and Producer.
Repository: flink Updated Branches: refs/heads/release-0.10 e977d8274 -> 9defe0cb2 [FLINK-3043] [docs] Fix description of Kafka Consumer and Producer. This also adds to the deprecated classes pointers forward to the designated classes. This closes #1380 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9defe0cb Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9defe0cb Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9defe0cb Branch: refs/heads/release-0.10 Commit: 9defe0cb289bfd8c3cfc37e8895e337147e82d29 Parents: 7441799 Author: Stephan EwenAuthored: Wed Nov 18 20:30:05 2015 +0100 Committer: Stephan Ewen Committed: Thu Nov 19 14:49:00 2015 +0100 -- docs/apis/kafka.md | 63 -- docs/apis/streaming_guide.md| 121 ++- .../connectors/kafka/FlinkKafkaProducer.java| 5 +- .../connectors/kafka/api/KafkaSink.java | 2 + .../api/persistent/PersistentKafkaSource.java | 5 + 5 files changed, 47 insertions(+), 149 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/9defe0cb/docs/apis/kafka.md -- diff --git a/docs/apis/kafka.md b/docs/apis/kafka.md deleted file mode 100644 index 0c0790a..000 --- a/docs/apis/kafka.md +++ /dev/null @@ -1,63 +0,0 @@ -title: "Reading from Kafka" -is_beta: true - - - - - -Interact with [Apache Kafka](https://kafka.apache.org/) streams from Flink's APIs. - -* This will be replaced by the TOC -{:toc} - - -Kafka Connector - -### Background - -Flink provides special Kafka Connectors for reading and writing data to Kafka topics. -The Flink Kafka Consumer integrates with Flink's checkpointing mechanisms to provide different -processing guarantees (most importantly exactly-once guarantees). - -For exactly-once processing Flink can not rely on the auto-commit capabilities of the Kafka consumers. -The Kafka consumer might commit offsets to Kafka which have not been processed successfully. - -Flink provides different connector implementations for different use-cases and environments. - - - - -### How to read data from Kafka - - Choose appropriate package and class - -Please pick a package (maven artifact id) and class name for your use-case and environment. For most users, the `flink-connector-kafka-083` package and the `FlinkKafkaConsumer082` class are appropriate. - -| Package | Supported Since | Class | Kafka Version | Allows exactly once processing | Notes | -| - |-| -| -- | -- | -| flink-connector-kafka | 0.9, 0.10 | `KafkaSource` | 0.8.1, 0.8.2 | **No**, does not participate in checkpointing at all. | Uses the old, high level KafkaConsumer API, autocommits to ZK by Kafka | -| flink-connector-kafka | 0.9, 0.10 | `PersistentKafkaSource` | 0.8.1, 0.8.2 | **No**, does not guarantee exactly-once processing, element order or strict partition assignment | Uses the old, high level KafkaConsumer API, offsets are committed into ZK manually | -| flink-connector-kafka-083 | 0.9.1 0.10 | `FlinkKafkaConsumer081` | 0.8.1 | **yes** | Uses the [SimpleConsumer](https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example) API of Kafka internally. Offsets are committed to ZK manually | -| flink-connector-kafka-083 | 0.9.1 0.10 | `FlinkKafkaConsumer082` | 0.8.2 | **yes** | Uses the [SimpleConsumer](https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example) API of Kafka internally. Offsets are committed to ZK manually | - - http://git-wip-us.apache.org/repos/asf/flink/blob/9defe0cb/docs/apis/streaming_guide.md -- diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md index a5b8b5b..3b3fd4f 100644 --- a/docs/apis/streaming_guide.md +++ b/docs/apis/streaming_guide.md @@ -3367,74 +3367,43 @@ with connectors. This connector provides access to event streams served by [Apache Kafka](https://kafka.apache.org/). -Flink provides special Kafka Connectors for reading and writing data to Kafka topics. -The Flink Kafka Consumer integrates with Flink's checkpointing mechanisms to provide different -processing guarantees (most importantly exactly-once guarantees). - -For exactly-once processing Flink can not rely on the auto-commit capabilities of the Kafka consumers. -The Kafka consumer might commit offsets to Kafka which have not been processed successfully. +Flink provides special Kafka Connectors for reading and writing data from/to Kafka topics. +The Flink Kafka Consumer integrates with Flink's checkpointing
[1/6] flink git commit: [FLINK-2832] [tests] Hardens RandomSamplerTest
Repository: flink Updated Branches: refs/heads/master 65ee28c34 -> 297d75c2e [FLINK-2832] [tests] Hardens RandomSamplerTest Increase the level of significance from p=0.01 to p=0.001 and add retry annotations to random sampler tests. This should decrease the probability of failing random sampler tests. This closes #2076 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/297d75c2 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/297d75c2 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/297d75c2 Branch: refs/heads/master Commit: 297d75c2e043026ccc3744d587c9ebbbd81e7d4b Parents: e1b55f0 Author: Till RohrmannAuthored: Mon Jun 6 16:19:30 2016 +0200 Committer: Stephan Ewen Committed: Wed Jun 8 15:17:10 2016 +0200 -- .../api/java/sampling/RandomSamplerTest.java| 77 .../src/test/resources/log4j-test.properties| 28 +++ flink-java/src/test/resources/logback-test.xml | 35 + 3 files changed, 111 insertions(+), 29 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/297d75c2/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java -- diff --git a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java index 68f9154..228dd3a 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java @@ -20,14 +20,16 @@ package org.apache.flink.api.java.sampling; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.commons.math3.stat.inference.KolmogorovSmirnovTest; +import org.apache.flink.testutils.junit.RetryOnFailure; +import org.apache.flink.testutils.junit.RetryRule; import org.apache.flink.util.Preconditions; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Set; @@ -50,29 +52,37 @@ import static org.junit.Assert.assertTrue; * @see https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test;>Kolmogorov Smirnov test */ public class RandomSamplerTest { - private final static int SOURCE_SIZE = 1; - private static KolmogorovSmirnovTest ksTest; - private static List source; - private final static int DEFFAULT_PARTITION_NUMBER=10; - private List[] sourcePartitions = new List[DEFFAULT_PARTITION_NUMBER]; + + private static final int SOURCE_SIZE = 1; + + private static final int DEFAULT_PARTITION_NUMBER = 10; + + private static final KolmogorovSmirnovTest ksTest = new KolmogorovSmirnovTest(); + + private static final List source = new ArrayList(SOURCE_SIZE); + + + @Rule + public final RetryRule retryRule = new RetryRule(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + private final List[] sourcePartitions = new List[DEFAULT_PARTITION_NUMBER]; + @BeforeClass public static void init() { // initiate source data set. - source = new ArrayList(SOURCE_SIZE); for (int i = 0; i < SOURCE_SIZE; i++) { source.add((double) i); } - - ksTest = new KolmogorovSmirnovTest(); } private void initSourcePartition() { - for (int i=0; i
[2/6] flink git commit: [hotfix] [nifi] Minor style cleanups in NiFi source
[hotfix] [nifi] Minor style cleanups in NiFi source Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b08b64ab Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b08b64ab Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b08b64ab Branch: refs/heads/master Commit: b08b64abdb7c9bd7946e9c36e63ec368a1ac5032 Parents: 38362c4 Author: Stephan EwenAuthored: Tue Jun 7 19:23:56 2016 +0200 Committer: Stephan Ewen Committed: Wed Jun 8 15:17:10 2016 +0200 -- .../streaming/connectors/nifi/NiFiSource.java | 99 ++-- 1 file changed, 49 insertions(+), 50 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/b08b64ab/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java -- diff --git a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java index 00b6921..57c59ec 100644 --- a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java +++ b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.connectors.nifi; import org.apache.flink.api.common.functions.StoppableFunction; @@ -26,6 +27,7 @@ import org.apache.nifi.remote.client.SiteToSiteClient; import org.apache.nifi.remote.client.SiteToSiteClientConfig; import org.apache.nifi.remote.protocol.DataPacket; import org.apache.nifi.stream.io.StreamUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,13 +42,20 @@ import java.util.Map; */ public class NiFiSource extends RichParallelSourceFunction implements StoppableFunction{ + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(NiFiSource.class); private static final long DEFAULT_WAIT_TIME_MS = 1000; - private long waitTimeMs; - private SiteToSiteClient client; - private SiteToSiteClientConfig clientConfig; + // + + private final SiteToSiteClientConfig clientConfig; + + private final long waitTimeMs; + + private transient SiteToSiteClient client; + private volatile boolean isRunning = true; /** @@ -73,63 +82,58 @@ public class NiFiSource extends RichParallelSourceFunction imple public void open(Configuration parameters) throws Exception { super.open(parameters); client = new SiteToSiteClient.Builder().fromConfig(clientConfig).build(); - isRunning = true; } @Override public void run(SourceContext ctx) throws Exception { - try { - while (isRunning) { - final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE); - if (transaction == null) { - LOG.warn("A transaction could not be created, waiting and will try again..."); - try { - Thread.sleep(waitTimeMs); - } catch (InterruptedException ignored) { - - } - continue; + while (isRunning) { + final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE); + if (transaction == null) { + LOG.warn("A transaction could not be created, waiting and will try again..."); + try { + Thread.sleep(waitTimeMs); + } catch (InterruptedException ignored) { + } + continue; + } - DataPacket dataPacket = transaction.receive(); - if (dataPacket == null) { - transaction.confirm(); - transaction.complete(); + DataPacket dataPacket = transaction.receive(); + if
[6/6] flink git commit: [FLINK-3405] [nifi] Extend NiFiSource with interface StoppableFunction
[FLINK-3405] [nifi] Extend NiFiSource with interface StoppableFunction This closes #2047 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/38362c40 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/38362c40 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/38362c40 Branch: refs/heads/master Commit: 38362c40bdd2e9a350630988c534b2859854d379 Parents: 6afb2b0 Author: smarthiAuthored: Sun May 29 02:15:16 2016 -0400 Committer: Stephan Ewen Committed: Wed Jun 8 15:17:10 2016 +0200 -- .../flink-connector-nifi/pom.xml| 2 +- .../streaming/connectors/nifi/NiFiSource.java | 24 ++-- 2 files changed, 18 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/38362c40/flink-streaming-connectors/flink-connector-nifi/pom.xml -- diff --git a/flink-streaming-connectors/flink-connector-nifi/pom.xml b/flink-streaming-connectors/flink-connector-nifi/pom.xml index d93bce7..a18d7b9 100644 --- a/flink-streaming-connectors/flink-connector-nifi/pom.xml +++ b/flink-streaming-connectors/flink-connector-nifi/pom.xml @@ -37,7 +37,7 @@ under the License. - 0.3.0 + 0.6.1 http://git-wip-us.apache.org/repos/asf/flink/blob/38362c40/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java -- diff --git a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java index a213bb4..00b6921 100644 --- a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java +++ b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java @@ -17,6 +17,7 @@ */ package org.apache.flink.streaming.connectors.nifi; +import org.apache.flink.api.common.functions.StoppableFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.nifi.remote.Transaction; @@ -37,7 +38,7 @@ import java.util.Map; * A source that pulls data from Apache NiFi using the NiFi Site-to-Site client. This source * produces NiFiDataPackets which encapsulate the content and attributes of a NiFi FlowFile. */ -public class NiFiSource extends RichParallelSourceFunction { +public class NiFiSource extends RichParallelSourceFunction implements StoppableFunction{ private static final Logger LOG = LoggerFactory.getLogger(NiFiSource.class); @@ -46,7 +47,7 @@ public class NiFiSource extends RichParallelSourceFunction { private long waitTimeMs; private SiteToSiteClient client; private SiteToSiteClientConfig clientConfig; - private transient volatile boolean running; + private volatile boolean isRunning = true; /** * Constructs a new NiFiSource using the given client config and the default wait time of 1000 ms. @@ -72,19 +73,19 @@ public class NiFiSource extends RichParallelSourceFunction { public void open(Configuration parameters) throws Exception { super.open(parameters); client = new SiteToSiteClient.Builder().fromConfig(clientConfig).build(); - running = true; + isRunning = true; } @Override public void run(SourceContext ctx) throws Exception { try { - while (running) { + while (isRunning) { final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE); if (transaction == null) { LOG.warn("A transaction could not be created, waiting and will try again..."); try { Thread.sleep(waitTimeMs); - } catch (InterruptedException e) { + } catch (InterruptedException ignored) { } continue; @@ -98,7 +99,7 @@ public class NiFiSource extends RichParallelSourceFunction { LOG.debug("No data available to pull, waiting and will try again...");
[3/6] flink git commit: [FLINK-4000] [RabbitMQ] Fix for checkpoint state restore at MessageAcknowledgingSourceBase
[FLINK-4000] [RabbitMQ] Fix for checkpoint state restore at MessageAcknowledgingSourceBase As says documentation for MessageAcknowledgingSourceBase.restoreState() This method is invoked when a function is executed as part of a recovery run. Note that restoreState() is called before open(). So current implementation 1. Fails on restoreState with NullPointerException, jobs fail to restart. 2. Does not restore anything because following open erases all checkpoint data immediately. 3. As consequence, violates exactly once rule because processed but not acknowledged list erased. Proposed change fixes that. This closes #2062 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ae679bb2 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ae679bb2 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ae679bb2 Branch: refs/heads/master Commit: ae679bb2aa1e0e239770605e049709fbc6b9962c Parents: 65ee28c Author: Alexey SavartsovAuthored: Thu Jun 2 02:23:53 2016 +0300 Committer: Stephan Ewen Committed: Wed Jun 8 15:17:10 2016 +0200 -- .../api/functions/source/MessageAcknowledgingSourceBase.java | 7 +-- 1 file changed, 5 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/ae679bb2/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java index d3cbfb6..9b2c4ac 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java @@ -127,8 +127,10 @@ public abstract class MessageAcknowledgingSourceBase @Override public void open(Configuration parameters) throws Exception { idsForCurrentCheckpoint = new ArrayList<>(64); - pendingCheckpoints = new ArrayDeque<>(numCheckpointsToKeep); - idsProcessedButNotAcknowledged = new HashSet<>(); + if (pendingCheckpoints == null) + pendingCheckpoints = new ArrayDeque<>(numCheckpointsToKeep); + if (idsProcessedButNotAcknowledged == null) + idsProcessedButNotAcknowledged = new HashSet<>(); } @Override @@ -177,6 +179,7 @@ public abstract class MessageAcknowledgingSourceBase @Override public void restoreState(SerializedCheckpointData[] state) throws Exception { + idsProcessedButNotAcknowledged = new HashSet<>(); pendingCheckpoints = SerializedCheckpointData.toDeque(state, idSerializer); // build a set which contains all processed ids. It may be used to check if we have // already processed an incoming message.
[4/6] flink git commit: [FLINK-4000] [RabbitMQ] Style cleanups in MessageAcknowledgingSourceBase
[FLINK-4000] [RabbitMQ] Style cleanups in MessageAcknowledgingSourceBase Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6afb2b00 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6afb2b00 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6afb2b00 Branch: refs/heads/master Commit: 6afb2b00f5438c176fe0579f632757722014e696 Parents: ae679bb Author: Stephan EwenAuthored: Tue Jun 7 19:20:03 2016 +0200 Committer: Stephan Ewen Committed: Wed Jun 8 15:17:10 2016 +0200 -- .../functions/source/MessageAcknowledgingSourceBase.java | 10 +- 1 file changed, 5 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6afb2b00/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java index 9b2c4ac..5c1b94e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java @@ -102,8 +102,6 @@ public abstract class MessageAcknowledgingSourceBase */ private transient Set idsProcessedButNotAcknowledged; - protected int numCheckpointsToKeep = 10; - // /** @@ -127,10 +125,12 @@ public abstract class MessageAcknowledgingSourceBase @Override public void open(Configuration parameters) throws Exception { idsForCurrentCheckpoint = new ArrayList<>(64); - if (pendingCheckpoints == null) - pendingCheckpoints = new ArrayDeque<>(numCheckpointsToKeep); - if (idsProcessedButNotAcknowledged == null) + if (pendingCheckpoints == null) { + pendingCheckpoints = new ArrayDeque<>(); + } + if (idsProcessedButNotAcknowledged == null) { idsProcessedButNotAcknowledged = new HashSet<>(); + } } @Override
[5/6] flink git commit: [FLINK-3922] [core] Fix case of infinite recursion in TypeExtractor
[FLINK-3922] [core] Fix case of infinite recursion in TypeExtractor This closes #2011 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/e1b55f03 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/e1b55f03 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/e1b55f03 Branch: refs/heads/master Commit: e1b55f033d18b22e8a3f07920fa7c9e5623d6922 Parents: b08b64a Author: twalthrAuthored: Thu May 19 17:01:57 2016 +0200 Committer: Stephan Ewen Committed: Wed Jun 8 15:17:10 2016 +0200 -- .../flink/api/java/typeutils/TypeExtractor.java | 2 +- .../java/typeutils/PojoTypeExtractionTest.java | 53 ++-- 2 files changed, 49 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/e1b55f03/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index 45420a2..9d30743 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -1245,7 +1245,7 @@ public class TypeExtractor { private static int countTypeInHierarchy(ArrayList typeHierarchy, Type type) { int count = 0; for (Type t : typeHierarchy) { - if (t == type || (isClassType(type) && t == typeToClass(type))) { + if (t == type || (isClassType(type) && t == typeToClass(type)) || (isClassType(t) && typeToClass(t) == type)) { count++; } } http://git-wip-us.apache.org/repos/asf/flink/blob/e1b55f03/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeExtractionTest.java -- diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeExtractionTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeExtractionTest.java index ffcfd52..16f3047 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeExtractionTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeExtractionTest.java @@ -90,6 +90,10 @@ public class PojoTypeExtractionTest { public String[] fancyArray; // generic type } + public static class FancyCollectionSubtype extends HashSet { + private static final long serialVersionUID = -3494469602638179921L; + } + public static class ParentSettingGenerics extends PojoWithGenerics { public String field3; } @@ -808,10 +812,49 @@ public class PojoTypeExtractionTest { Assert.assertTrue(tti.getTypeAt(0) instanceof PojoTypeInfo); Assert.assertTrue(tti.getTypeAt(1) instanceof PojoTypeInfo); } - - // - - public static class FancyCollectionSubtype extends HashSet { - private static final long serialVersionUID = -3494469602638179921L; + + public static class PojoWithRecursiveGenericField { + public PojoWithRecursiveGenericField parent; + public PojoWithRecursiveGenericField(){} + } + + @Test + public void testPojoWithRecursiveGenericField() { + TypeInformation ti = TypeExtractor.createTypeInfo(PojoWithRecursiveGenericField.class); + Assert.assertTrue(ti instanceof PojoTypeInfo); + Assert.assertEquals(GenericTypeInfo.class, ((PojoTypeInfo) ti).getPojoFieldAt(0).getTypeInformation().getClass()); + } + + public static class MutualPojoA { + public MutualPojoB field; + } + + public static class MutualPojoB { + public MutualPojoA field; + } + + @Test + public void testPojosWithMutualRecursion() { + TypeInformation ti = TypeExtractor.createTypeInfo(MutualPojoB.class); + Assert.assertTrue(ti instanceof PojoTypeInfo); + TypeInformation pti = ((PojoTypeInfo) ti).getPojoFieldAt(0).getTypeInformation(); + Assert.assertTrue(pti instanceof PojoTypeInfo); + Assert.assertEquals(GenericTypeInfo.class, ((PojoTypeInfo) pti).getPojoFieldAt(0).getTypeInformation().getClass()); } + + public static class Container { + public T
flink git commit: [hotfix] [core] Make sure Dropwizard reporters do not block operator creation/teardown while reporting
Repository: flink Updated Branches: refs/heads/master 393bceb58 -> 2bffd7b9e [hotfix] [core] Make sure Dropwizard reporters do not block operator creation/teardown while reporting The "report()" method used to hold the same lock that was needed to create and shutdown metrics. Holding that lock too long could delay operator creation and teardown. This also removes the no longer needed dependency on dropwizard in flink-core. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2bffd7b9 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2bffd7b9 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2bffd7b9 Branch: refs/heads/master Commit: 2bffd7b9e558b0ee7efb1bc23b1047faed48c4be Parents: 393bceb Author: Stephan EwenAuthored: Fri Jun 10 00:24:57 2016 +0200 Committer: Stephan Ewen Committed: Fri Jun 10 14:50:36 2016 +0200 -- flink-core/pom.xml | 6 -- .../dropwizard/ScheduledDropwizardReporter.java | 19 +++ 2 files changed, 11 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/2bffd7b9/flink-core/pom.xml -- diff --git a/flink-core/pom.xml b/flink-core/pom.xml index b64f94e..1aadfbd 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -47,12 +47,6 @@ under the License. - - io.dropwizard.metrics - metrics-core - ${metrics.version} - - org.apache.avro http://git-wip-us.apache.org/repos/asf/flink/blob/2bffd7b9/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java -- diff --git a/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java b/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java index 74fdb85..d67f3e3 100644 --- a/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java +++ b/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java @@ -35,6 +35,7 @@ import org.apache.flink.metrics.reporter.Scheduled; import java.util.HashMap; import java.util.Map; +import java.util.SortedMap; /** * Base class for {@link org.apache.flink.metrics.reporter.MetricReporter} that wraps a @@ -123,14 +124,16 @@ public abstract class ScheduledDropwizardReporter implements MetricReporter, Sch @Override public void report() { - synchronized (this) { - this.reporter.report( - this.registry.getGauges(), - this.registry.getCounters(), - this.registry.getHistograms(), - this.registry.getMeters(), - this.registry.getTimers()); - } + // we do not need to lock here, because the dropwizard registry is + // internally a concurrent map + @SuppressWarnings("rawtypes") + final SortedMap gauges = registry.getGauges(); + final SortedMap counters = registry.getCounters(); + final SortedMap histograms = registry.getHistograms(); + final SortedMap meters = registry.getMeters(); + final SortedMap timers = registry.getTimers(); + + this.reporter.report(gauges, counters, histograms, meters, timers); } public abstract ScheduledReporter getReporter(Configuration config);
flink git commit: [hotfix] [core] Re-enable JMX metrics as the default.
Repository: flink Updated Branches: refs/heads/master ccc86e9f1 -> d760bbc96 [hotfix] [core] Re-enable JMX metrics as the default. This also does some minor code cleanups, logging fixes, and smoother concurrency handling. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d760bbc9 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d760bbc9 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d760bbc9 Branch: refs/heads/master Commit: d760bbc962ea34983f3dad5e72fdb9d6a3d41c15 Parents: ccc86e9 Author: Stephan EwenAuthored: Wed Jun 8 21:02:54 2016 +0200 Committer: Stephan Ewen Committed: Thu Jun 9 21:14:56 2016 +0200 -- .../apache/flink/metrics/MetricRegistry.java| 12 --- .../flink/metrics/reporter/JMXReporter.java | 2 +- .../flink/metrics/statsd/StatsDReporter.java| 21 +++- .../flink/runtime/taskmanager/TaskManager.scala | 8 ++-- 4 files changed, 32 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/d760bbc9/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java -- diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java index 52a44cf..b3422e1 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java @@ -57,7 +57,7 @@ public class MetricRegistry { // configuration keys // - private static final Logger LOG = LoggerFactory.getLogger(MetricRegistry.class); + static final Logger LOG = LoggerFactory.getLogger(MetricRegistry.class); private final MetricReporter reporter; private final java.util.Timer timer; @@ -83,7 +83,9 @@ public class MetricRegistry { final String className = config.getString(KEY_METRICS_REPORTER_CLASS, null); if (className == null) { - this.reporter = null; + // by default, create JMX metrics + LOG.info("No metrics reporter configured, exposing metrics via JMX"); + this.reporter = new JMXReporter(); this.timer = null; } else { @@ -239,7 +241,11 @@ public class MetricRegistry { @Override public void run() { - reporter.report(); + try { + reporter.report(); + } catch (Throwable t) { + LOG.warn("Error while reporting metrics", t); + } } } } http://git-wip-us.apache.org/repos/asf/flink/blob/d760bbc9/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java -- diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java index 71c80de..db81164 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java @@ -23,8 +23,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Metric; - import org.apache.flink.metrics.groups.AbstractMetricGroup; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; http://git-wip-us.apache.org/repos/asf/flink/blob/d760bbc9/flink-metric-reporters/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java -- diff --git a/flink-metric-reporters/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java b/flink-metric-reporters/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java index 124b21d..ae57f55 100644 --- a/flink-metric-reporters/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java +++ b/flink-metric-reporters/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java @@ -33,7 +33,9 @@ import java.net.DatagramPacket; import java.net.DatagramSocket; import java.net.InetSocketAddress; import java.net.SocketException; +import java.util.ConcurrentModificationException; import
flink git commit: [FLINK-4052] [runtime] Improve test stability for ConnectionUtilsTest
Repository: flink Updated Branches: refs/heads/master d760bbc96 -> 8696e799f [FLINK-4052] [runtime] Improve test stability for ConnectionUtilsTest Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8696e799 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8696e799 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8696e799 Branch: refs/heads/master Commit: 8696e799faf18027cd9f2479b24520cdcbca8bdb Parents: d760bbc Author: Stephan EwenAuthored: Thu Jun 9 21:32:45 2016 +0200 Committer: Stephan Ewen Committed: Thu Jun 9 21:32:45 2016 +0200 -- .../flink/runtime/net/ConnectionUtilsTest.java | 20 ++-- 1 file changed, 6 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/8696e799/flink-runtime/src/test/java/org/apache/flink/runtime/net/ConnectionUtilsTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/net/ConnectionUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/net/ConnectionUtilsTest.java index 57360ff..7d615bd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/net/ConnectionUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/net/ConnectionUtilsTest.java @@ -24,6 +24,7 @@ import org.junit.Test; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.ServerSocket; /** * Tests for the network utilities. @@ -32,20 +33,11 @@ public class ConnectionUtilsTest { @Test public void testReturnLocalHostAddressUsingHeuristics() { - int unusedPort; - try { - unusedPort = org.apache.flink.util.NetUtils.getAvailablePort(); - } - catch (Throwable t) { - // if this system cannot find an available port, - // skip this test - return; - } - - try { - // create an unreachable target address - InetSocketAddress unreachable = new InetSocketAddress("localhost", unusedPort); - + try (ServerSocket blocker = new ServerSocket(0, 1, InetAddress.getLocalHost())) { + // the "blocker" server socket simply does not accept connections + // this address is consequently "unreachable" + InetSocketAddress unreachable = new InetSocketAddress("localhost", blocker.getLocalPort()); + final long start = System.currentTimeMillis(); InetAddress add = ConnectionUtils.findConnectingAddress(unreachable, 2000, 400);
flink git commit: [hotfix] [core] Fix scope format keys for metrics
Repository: flink Updated Branches: refs/heads/master 65545c2ed -> ccc3e44cb [hotfix] [core] Fix scope format keys for metrics This closes #2068 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ccc3e44c Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ccc3e44c Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ccc3e44c Branch: refs/heads/master Commit: ccc3e44cb3ccb76275dd5a72afe285242b0127b9 Parents: 65545c2 Author: zentolAuthored: Fri Jun 3 16:07:21 2016 +0200 Committer: Stephan Ewen Committed: Mon Jun 6 00:22:22 2016 +0200 -- .../main/java/org/apache/flink/metrics/MetricRegistry.java | 8 1 file changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/ccc3e44c/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java -- diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java index 4efdbab..52a44cf 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java @@ -48,10 +48,10 @@ public class MetricRegistry { public static final String KEY_METRICS_REPORTER_ARGUMENTS = "metrics.reporter.arguments"; public static final String KEY_METRICS_REPORTER_INTERVAL = "metrics.reporter.interval"; - public static final String KEY_METRICS_SCOPE_NAMING_TM = "metrics.scopeName.tm"; - public static final String KEY_METRICS_SCOPE_NAMING_JOB = "metrics.scopeName.job"; - public static final String KEY_METRICS_SCOPE_NAMING_TASK = "metrics.scopeName.task"; - public static final String KEY_METRICS_SCOPE_NAMING_OPERATOR = "metrics.scopeName.operator"; + public static final String KEY_METRICS_SCOPE_NAMING_TM = "metrics.scope.tm"; + public static final String KEY_METRICS_SCOPE_NAMING_JOB = "metrics.scope.job"; + public static final String KEY_METRICS_SCOPE_NAMING_TASK = "metrics.scope.task"; + public static final String KEY_METRICS_SCOPE_NAMING_OPERATOR = "metrics.scope.operator"; // // configuration keys
[3/3] flink git commit: [hotfix] Remove leftover config key constant from ExecutionConfig
[hotfix] Remove leftover config key constant from ExecutionConfig Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/5a7f4e33 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/5a7f4e33 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/5a7f4e33 Branch: refs/heads/master Commit: 5a7f4e3395bfb06da496584be88501c328f6ac1d Parents: ef5832d Author: Stephan EwenAuthored: Wed May 25 16:20:27 2016 +0200 Committer: Stephan Ewen Committed: Thu May 26 21:15:20 2016 +0200 -- .../main/java/org/apache/flink/api/common/ExecutionConfig.java| 3 --- 1 file changed, 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/5a7f4e33/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index d27760f..86d3be6 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -61,9 +61,6 @@ public class ExecutionConfig implements Serializable { private static final long serialVersionUID = 1L; - // Key for storing it in the Job Configuration - public static final String CONFIG_KEY = "runtime.config"; - /** * The constant to use for the parallelism, if the system should use the number * of currently available slots.
[1/3] flink git commit: [FLINK-3962] [core] Properly initialize I/O Metric Group
Repository: flink Updated Branches: refs/heads/master ef5832d8f -> 573a92fc5 [FLINK-3962] [core] Properly initialize I/O Metric Group Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/573a92fc Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/573a92fc Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/573a92fc Branch: refs/heads/master Commit: 573a92fc5fe95aa7b07f5f7dc16c3d2a8efba16e Parents: bf256c7 Author: Stephan EwenAuthored: Wed May 25 18:39:16 2016 +0200 Committer: Stephan Ewen Committed: Thu May 26 21:15:20 2016 +0200 -- .../java/org/apache/flink/metrics/groups/TaskMetricGroup.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/573a92fc/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java -- diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java index 3238534..316c84f 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java @@ -65,12 +65,13 @@ public class TaskMetricGroup extends ComponentMetricGroup { this.executionId = executionId; this.subtaskIndex = subtaskIndex; - this.ioMetrics = new IOMetricGroup(registry, this); this.formats.put(SCOPE_TASK_ID, taskId.toString()); this.formats.put(SCOPE_TASK_ATTEMPT, executionId.toString()); this.formats.put(SCOPE_TASK_NAME, checkNotNull(name)); this.formats.put(SCOPE_TASK_SUBTASK_INDEX, String.valueOf(subtaskIndex)); + + this.ioMetrics = new IOMetricGroup(registry, this); } public OperatorMetricGroup addOperator(String name) {
[2/3] flink git commit: [hotfix] Fix access to temp file directories in SpillingAdaptiveSpanningRecordDeserializer
[hotfix] Fix access to temp file directories in SpillingAdaptiveSpanningRecordDeserializer Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/bf256c7f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/bf256c7f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/bf256c7f Branch: refs/heads/master Commit: bf256c7fbe05accdadc8470013879f567341d1aa Parents: 5a7f4e3 Author: Stephan EwenAuthored: Wed May 25 17:04:31 2016 +0200 Committer: Stephan Ewen Committed: Thu May 26 21:15:20 2016 +0200 -- .../api/common/functions/RuntimeContext.java| 2 +- .../org/apache/flink/metrics/MetricGroup.java | 6 +-- .../clusterframework/types/ResourceID.java | 34 +++ .../runtime/io/disk/iomanager/IOManager.java| 15 ++- .../api/reader/AbstractRecordReader.java| 12 +- .../network/api/reader/MutableRecordReader.java | 12 +- .../io/network/api/reader/RecordReader.java | 17 +--- ...llingAdaptiveSpanningRecordDeserializer.java | 16 ++- .../task/IterationSynchronizationSinkTask.java | 6 ++- .../flink/runtime/operators/BatchTask.java | 18 +--- .../flink/runtime/operators/DataSinkTask.java | 8 +++- .../taskmanager/TaskManagerRuntimeInfo.java | 35 ++-- .../flink/runtime/taskmanager/TaskManager.scala | 3 +- .../SpanningRecordSerializationTest.java| 4 +- .../network/serialization/LargeRecordsTest.java | 5 ++- .../SlotCountExceedingParallelismTest.java | 5 ++- .../operators/drivers/TestTaskContext.java | 3 +- .../testutils/BinaryOperatorTestBase.java | 3 +- .../operators/testutils/DriverTestBase.java | 3 +- .../operators/testutils/MockEnvironment.java| 5 ++- .../testutils/UnaryOperatorTestBase.java| 3 +- .../runtime/taskmanager/TaskAsyncCallTest.java | 2 +- .../runtime/taskmanager/TaskCancelTest.java | 10 +++-- .../flink/runtime/taskmanager/TaskTest.java | 2 +- .../apache/flink/runtime/jobmanager/Tasks.scala | 44 .../runtime/io/StreamInputProcessor.java| 3 +- .../runtime/io/StreamTwoInputProcessor.java | 3 +- .../runtime/tasks/StreamMockEnvironment.java| 4 +- .../streaming/runtime/tasks/StreamTaskTest.java | 2 +- .../runtime/NetworkStackThroughputITCase.java | 20 ++--- 30 files changed, 212 insertions(+), 93 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/bf256c7f/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index ed2f613..9a04b24 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -63,7 +63,7 @@ public interface RuntimeContext { * Returns the metric group for this parallel subtask. * * @return The metric group for this parallel subtask. - */ +*/ MetricGroup getMetricGroup(); /** http://git-wip-us.apache.org/repos/asf/flink/blob/bf256c7f/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java -- diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java index a3832ff..6c9e044 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java @@ -21,14 +21,14 @@ package org.apache.flink.metrics; import org.apache.flink.annotation.PublicEvolving; /** - * A MetricGroup is a named container for {@link Metric Metrics} and {@link MetricGroup MetricGroups}. + * A MetricGroup is a named container for {@link Metric Metrics} and further metric subgroups. * * Instances of this class can be used to register new metrics with Flink and to create a nested * hierarchy based on the group names. * * A MetricGroup is uniquely identified by it's place in the hierarchy and name. * - * Metrics groups can be {@link #close() closed}. Upon closing, they de-register all metrics + * Metrics groups can be {@link #close() closed}. Upon closing, the group de-register all metrics * from any metrics reporter and any internal maps. Note that even closed metrics groups * return Counters, Gauges, etc to the code, to prevent exceptions in the monitored code. * These metrics simply do not get reported any more, when
[7/7] flink git commit: [hotfix] [yarn tests] Fix deadlock between YARN Session CLI tests and Surefire
[hotfix] [yarn tests] Fix deadlock between YARN Session CLI tests and Surefire The Surefire Plugin uses stdin to communicate with forked JVMs for tests. The YARN Session CLI tests also try to read the stdin stream. The tests deadlock since Surefire never releases the stdin locks during the lifetime of a test. This change adds a parameter whether the YARN Session CLI should try to read user console input, and sets this to false in the integration tests. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/da23ee38 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/da23ee38 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/da23ee38 Branch: refs/heads/master Commit: da23ee38e5b36ddf26a6a5a807efbbbcbfe1d517 Parents: 6511557 Author: Stephan EwenAuthored: Mon May 30 19:49:25 2016 +0200 Committer: Stephan Ewen Committed: Mon May 30 19:49:25 2016 +0200 -- .../flink/client/FlinkYarnSessionCli.java | 24 +--- .../flink/client/cli/CliFrontendParser.java | 2 +- .../flink/yarn/FlinkYarnSessionCliTest.java | 2 +- .../org/apache/flink/yarn/YarnTestBase.java | 2 +- .../src/test/resources/log4j-test.properties| 3 ++- 5 files changed, 21 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/da23ee38/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java -- diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java index 91f8df2..bb61ffb 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java @@ -82,6 +82,8 @@ public class FlinkYarnSessionCli { */ private final Option DYNAMIC_PROPERTIES; + private final boolean acceptInteractiveInput; + // Internal fields - private AbstractFlinkYarnCluster yarnCluster = null; private boolean detachedMode = false; @@ -89,7 +91,9 @@ public class FlinkYarnSessionCli { /** Default yarn application name. */ private String defaultApplicationName = null; - public FlinkYarnSessionCli(String shortPrefix, String longPrefix) { + public FlinkYarnSessionCli(String shortPrefix, String longPrefix, boolean acceptInteractiveInput) { + this.acceptInteractiveInput = acceptInteractiveInput; + QUERY = new Option(shortPrefix + "q", longPrefix + "query", false, "Display available YARN resources (memory, cores)"); QUEUE = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue."); SHIP_PATH = new Option(shortPrefix + "t", longPrefix + "ship", true, "Ship files in the specified directory (t for transfer)"); @@ -292,7 +296,7 @@ public class FlinkYarnSessionCli { propertiesFile.setReadable(true, false); // readable for all. } - public static void runInteractiveCli(AbstractFlinkYarnCluster yarnCluster) { + public static void runInteractiveCli(AbstractFlinkYarnCluster yarnCluster, boolean readConsoleInput) { final String HELP = "Available commands:\n" + "help - show these commands\n" + "stop - stop the YARN session"; @@ -304,6 +308,8 @@ public class FlinkYarnSessionCli { // -- check if there are updates by the cluster --- GetClusterStatusResponse status = yarnCluster.getClusterStatus(); + LOG.debug("Received status message: {}", status); + if (status != null && numTaskmanagers != status.numRegisteredTaskManagers()) { System.err.println("Number of connected TaskManagers changed to " + status.numRegisteredTaskManagers() + ". " + @@ -324,15 +330,16 @@ public class FlinkYarnSessionCli { yarnCluster.shutdown(true); } - // wait until CLIENT_POLLING_INTERVALL is over or the user entered something. + // wait until CLIENT_POLLING_INTERVAL is over or the user entered something. long startTime = System.currentTimeMillis(); while ((System.currentTimeMillis() - startTime) <
[5/7] flink git commit: [FLINK-1502] [core] Cleanups, robustness, and performance improvements in the metrics system
[FLINK-1502] [core] Cleanups, robustness, and performance improvements in the metrics system Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/7ad8375a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7ad8375a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7ad8375a Branch: refs/heads/master Commit: 7ad8375a89374bec80571029e9166f1336bdea8e Parents: d3e3bd5 Author: Stephan EwenAuthored: Wed May 25 20:34:44 2016 +0200 Committer: Stephan Ewen Committed: Mon May 30 14:54:10 2016 +0200 -- flink-contrib/flink-storm/pom.xml | 22 +- .../flink/storm/wrappers/BoltWrapperTest.java | 13 +- .../common/operators/CollectionExecutor.java| 32 +- .../java/org/apache/flink/metrics/Counter.java | 5 +- .../java/org/apache/flink/metrics/Gauge.java| 3 +- .../java/org/apache/flink/metrics/Metric.java | 3 +- .../apache/flink/metrics/MetricRegistry.java| 92 ++--- .../metrics/groups/AbstractMetricGroup.java | 98 +++-- .../metrics/groups/ComponentMetricGroup.java| 109 + .../metrics/groups/GenericMetricGroup.java | 45 +-- .../flink/metrics/groups/IOMetricGroup.java | 27 +- .../flink/metrics/groups/JobMetricGroup.java| 88 ++-- .../groups/NonRegisteringMetricsGroup.java | 87 .../metrics/groups/OperatorMetricGroup.java | 35 +- .../org/apache/flink/metrics/groups/Scope.java | 119 -- .../metrics/groups/TaskManagerMetricGroup.java | 68 ++-- .../flink/metrics/groups/TaskMetricGroup.java | 126 -- .../groups/UnregisteredMetricsGroup.java| 73 .../flink/metrics/groups/scope/ScopeFormat.java | 399 +++ .../metrics/groups/scope/ScopeFormats.java | 105 + .../metrics/reporter/AbstractReporter.java | 40 +- .../flink/metrics/reporter/JMXReporter.java | 194 ++--- .../flink/metrics/reporter/MetricReporter.java | 53 +-- .../flink/metrics/reporter/Scheduled.java | 8 +- .../functions/util/RuntimeUDFContextTest.java | 33 +- .../api/common/io/RichInputFormatTest.java | 18 +- .../api/common/io/RichOutputFormatTest.java | 19 +- .../operators/GenericDataSinkBaseTest.java | 16 +- .../operators/GenericDataSourceBaseTest.java| 18 +- .../base/FlatMapOperatorCollectionTest.java | 9 +- .../base/InnerJoinOperatorBaseTest.java | 15 +- .../common/operators/base/MapOperatorTest.java | 36 +- .../base/PartitionMapOperatorTest.java | 19 +- .../flink/metrics/MetricRegistryTest.java | 78 +--- .../flink/metrics/groups/JobGroupTest.java | 73 ++-- .../groups/MetricGroupRegistrationTest.java | 7 +- .../flink/metrics/groups/MetricGroupTest.java | 32 +- .../flink/metrics/groups/OperatorGroupTest.java | 66 +-- .../flink/metrics/groups/TaskGroupTest.java | 103 +++-- .../metrics/groups/TaskManagerGroupTest.java| 58 +-- .../flink/metrics/reporter/JMXReporterTest.java | 35 +- .../flink/metrics/util/DummyJobMetricGroup.java | 47 --- .../flink/metrics/util/DummyMetricGroup.java| 57 --- .../flink/metrics/util/DummyMetricRegistry.java | 29 -- .../metrics/util/DummyOperatorMetricGroup.java | 37 -- .../flink/metrics/util/DummyReporter.java | 47 --- .../util/DummyTaskManagerMetricGroup.java | 42 -- .../metrics/util/DummyTaskMetricGroup.java | 42 -- .../apache/flink/metrics/util/TestReporter.java | 11 +- .../base/CoGroupOperatorCollectionTest.java | 6 +- .../operators/base/GroupReduceOperatorTest.java | 17 +- .../base/InnerJoinOperatorBaseTest.java | 24 +- .../operators/base/ReduceOperatorTest.java | 23 +- .../dropwizard/ScheduledDropwizardReporter.java | 104 +++-- .../flink/dropwizard/metrics/GaugeWrapper.java | 8 + .../flink/metrics/ganglia/GangliaReporter.java | 79 .../flink/metrics/graphite/GangliaReporter.java | 73 .../metrics/graphite/GraphiteReporter.java | 17 +- .../flink/metrics/statsd/StatsDReporter.java| 79 ++-- .../flink/runtime/taskmanager/TaskManager.scala | 3 +- .../operators/drivers/TestTaskContext.java | 4 +- .../testutils/BinaryOperatorTestBase.java | 6 +- .../operators/testutils/DriverTestBase.java | 31 +- .../operators/testutils/DummyEnvironment.java | 28 +- .../operators/testutils/MockEnvironment.java| 3 +- .../testutils/UnregisteredTaskMetricsGroup.java | 68 ...AlignedProcessingTimeWindowOperatorTest.java | 20 +- ...AlignedProcessingTimeWindowOperatorTest.java | 14 +- .../runtime/tasks/StreamMockEnvironment.java| 4 +- 69 files changed, 1893 insertions(+), 1509 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/7ad8375a/flink-contrib/flink-storm/pom.xml
[1/7] flink git commit: [hotfix] [contrib] Fix robustness of DataStreamUtils stream collecting
Repository: flink Updated Branches: refs/heads/master 035f62969 -> da23ee38e [hotfix] [contrib] Fix robustness of DataStreamUtils stream collecting Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d3e3bd52 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d3e3bd52 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d3e3bd52 Branch: refs/heads/master Commit: d3e3bd528ed972680baebda05acce0bf39af6798 Parents: 77afe28 Author: Stephan EwenAuthored: Thu May 26 20:23:20 2016 +0200 Committer: Stephan Ewen Committed: Mon May 30 14:41:34 2016 +0200 -- .../flink/contrib/streaming/CollectSink.java| 118 .../contrib/streaming/DataStreamIterator.java | 134 - .../contrib/streaming/DataStreamUtils.java | 65 --- .../contrib/streaming/SocketStreamIterator.java | 190 +++ .../flink/contrib/streaming/CollectITCase.java | 48 ++--- .../streaming/SocketStreamIteratorTest.java | 104 ++ 6 files changed, 398 insertions(+), 261 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/d3e3bd52/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/CollectSink.java -- diff --git a/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/CollectSink.java b/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/CollectSink.java index 53b84ef..161eb16 100644 --- a/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/CollectSink.java +++ b/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/CollectSink.java @@ -17,35 +17,30 @@ package org.apache.flink.contrib.streaming; -import java.io.DataOutputStream; import java.io.IOException; import java.io.OutputStream; import java.net.Socket; import java.net.InetAddress; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.api.functions.sink.SocketClientSink; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataOutputView; /** * A specialized data sink to be used by DataStreamUtils.collect. */ class CollectSink extends RichSinkFunction { + private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(SocketClientSink.class); - private final InetAddress hostIp; private final int port; private final TypeSerializer serializer; + private transient Socket client; - private transient DataOutputStream dataOutputStream; - private StreamWriterDataOutputView streamWriter; + private transient OutputStream outputStream; + private transient DataOutputViewStreamWrapper streamWriter; /** * Creates a CollectSink that will send the data to the specified host. @@ -60,57 +55,13 @@ class CollectSink extends RichSinkFunction { this.serializer = serializer; } - /** -* Initializes the connection to Socket. -*/ - public void initializeConnection() { - OutputStream outputStream; - try { - client = new Socket(hostIp, port); - outputStream = client.getOutputStream(); - streamWriter = new StreamWriterDataOutputView(outputStream); - } catch (IOException e) { - throw new RuntimeException(e); - } - dataOutputStream = new DataOutputStream(outputStream); - } - - /** -* Called when new data arrives to the sink, and forwards it to Socket. -* -* @param value -* The incoming data -*/ @Override - public void invoke(IN value) { + public void invoke(IN value) throws Exception { try { serializer.serialize(value, streamWriter); - } catch (IOException e) { - if(LOG.isErrorEnabled()){ - LOG.error("Cannot send message to socket server at " + hostIp.toString() + ":" + port, e); - } } - } - - /** -* Closes the connection of the Socket client. -*/ - private void closeConnection(){ -
[4/4] flink git commit: [FLINK-3886] [clients] Give a better error when the application Main class is not public.
[FLINK-3886] [clients] Give a better error when the application Main class is not public. This closes #2043 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6db9e6ae Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6db9e6ae Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6db9e6ae Branch: refs/heads/master Commit: 6db9e6ae5c2f4968687a3fcae0520e05442bd9ab Parents: bcf5f46 Author: Niels BasjesAuthored: Fri May 27 13:51:04 2016 +0200 Committer: Stephan Ewen Committed: Tue May 31 16:39:56 2016 +0200 -- .../apache/flink/client/program/PackagedProgram.java| 4 .../org/apache/flink/client/program/ClientTest.java | 12 ++-- 2 files changed, 10 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6db9e6ae/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java -- diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java index f78502a..40092d8 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java @@ -484,6 +484,10 @@ public class PackagedProgram { private static void callMainMethod(Class entryClass, String[] args) throws ProgramInvocationException { Method mainMethod; + if (!Modifier.isPublic(entryClass.getModifiers())) { + throw new ProgramInvocationException("The class " + entryClass.getName() + " must be public."); + } + try { mainMethod = entryClass.getMethod("main", String[].class); } catch (NoSuchMethodException e) { http://git-wip-us.apache.org/repos/asf/flink/blob/6db9e6ae/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java -- diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java index cc32d9c..4f9b367 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java @@ -378,7 +378,7 @@ public class ClientTest { } } - private static final class TestExecuteTwice { + public static final class TestExecuteTwice { public static void main(String args[]) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -388,7 +388,7 @@ public class ClientTest { } } - private static final class TestEager { + public static final class TestEager { public static void main(String args[]) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -396,7 +396,7 @@ public class ClientTest { } } - private static final class TestGetRuntime { + public static final class TestGetRuntime { public static void main(String args[]) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -405,7 +405,7 @@ public class ClientTest { } } - private static final class TestGetJobID { + public static final class TestGetJobID { public static void main(String args[]) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -414,7 +414,7 @@ public class ClientTest { } } - private static final class TestGetAccumulator { + public static final class TestGetAccumulator { public static void main(String args[]) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -423,7 +423,7 @@ public class ClientTest { } } - private static final class TestGetAllAccumulator { + public static final class TestGetAllAccumulator { public static void main(String args[]) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
[3/4] flink git commit: [FLINK-3979] [docs] Add import statements for last code sample in the quickstart docs.
[FLINK-3979] [docs] Add import statements for last code sample in the quickstart docs. This closes #2038 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ad52a5f0 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ad52a5f0 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ad52a5f0 Branch: refs/heads/master Commit: ad52a5f0cebe6e5d593be0ae40656b28470af106 Parents: 5b02879 Author: Zhai JiaAuthored: Thu May 26 23:24:40 2016 +0800 Committer: Stephan Ewen Committed: Tue May 31 16:39:56 2016 +0200 -- docs/quickstart/run_example_quickstart.md | 9 - 1 file changed, 8 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/ad52a5f0/docs/quickstart/run_example_quickstart.md -- diff --git a/docs/quickstart/run_example_quickstart.md b/docs/quickstart/run_example_quickstart.md index 0d5e919..0939297 100644 --- a/docs/quickstart/run_example_quickstart.md +++ b/docs/quickstart/run_example_quickstart.md @@ -315,6 +315,13 @@ result .addSink(new FlinkKafkaProducer08<>("localhost:9092", "wiki-result", new SimpleStringSchema())); {% endhighlight %} +The related classes also need to be imported: +{% highlight java %} +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer08; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.apache.flink.api.common.functions.MapFunction; +{% endhighlight %} + Note how we first transform the Stream of `Tuple2 ` to a Stream of `String` using a MapFunction. We are doing this because it is easier to write plain strings to Kafka. Then, we create a Kafka sink. You might have to adapt the hostname and port to your setup. `"wiki-result"` @@ -382,4 +389,4 @@ and, for example, see the number of processed elements: -This concludes our little tour of Flink. If you have any questions, please don't hesitate to ask on our [Mailing Lists](http://flink.apache.org/community.html#mailing-lists). \ No newline at end of file +This concludes our little tour of Flink. If you have any questions, please don't hesitate to ask on our [Mailing Lists](http://flink.apache.org/community.html#mailing-lists).
[2/4] flink git commit: [FLINK-3978] [core] Add hasBroadcastVariable method to RuntimeContext
[FLINK-3978] [core] Add hasBroadcastVariable method to RuntimeContext New method RuntimeContext.hasBroadcastVariable(String). Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/5b028797 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/5b028797 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/5b028797 Branch: refs/heads/master Commit: 5b0287971fa2beda360105d96e7bfbc7a110fae7 Parents: 6db9e6a Author: Greg HoganAuthored: Thu May 26 14:45:00 2016 -0400 Committer: Stephan Ewen Committed: Tue May 31 16:39:56 2016 +0200 -- .../api/common/functions/RuntimeContext.java | 10 ++ .../common/functions/util/RuntimeUDFContext.java | 5 + .../functions/util/RuntimeUDFContextTest.java | 9 +++-- .../util/DistributedRuntimeUDFContext.java| 18 +- .../api/operators/StreamingRuntimeContext.java| 5 + 5 files changed, 36 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/5b028797/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index 72e1a4d..e409c11 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -164,6 +164,16 @@ public interface RuntimeContext { // /** +* Tests for the existence of the broadcast variable identified by the +* given {@code name}. +* +* @param name The name under which the broadcast variable is registered; +* @return Whether a broadcast variable exists for the given name. +*/ + @PublicEvolving + boolean hasBroadcastVariable(String name); + + /** * Returns the result bound to the broadcast variable identified by the * given {@code name}. * http://git-wip-us.apache.org/repos/asf/flink/blob/5b028797/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java index 6571d0d..ba3f85e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java @@ -49,6 +49,11 @@ public class RuntimeUDFContext extends AbstractRuntimeUDFContext { } @Override + public boolean hasBroadcastVariable(String name) { + return this.initializedBroadcastVars.containsKey(name) || this.uninitializedBroadcastVars.containsKey(name); + } + + @Override @SuppressWarnings("unchecked") public List getBroadcastVariable(String name) { http://git-wip-us.apache.org/repos/asf/flink/blob/5b028797/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java -- diff --git a/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java b/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java index 8f00cd5..4cd2a64 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java @@ -48,7 +48,9 @@ public class RuntimeUDFContextTest { new HashMap (), new HashMap (), new UnregisteredMetricsGroup()); - + + assertFalse(ctx.hasBroadcastVariable("some name")); + try { ctx.getBroadcastVariable("some name"); fail("should throw an exception"); @@ -85,7 +87,10 @@ public class RuntimeUDFContextTest { ctx.setBroadcastVariable("name1", Arrays.asList(1, 2, 3, 4)); ctx.setBroadcastVariable("name2",
flink git commit: [hotfix] Use AccumulatorSnapshot's class loader for deserializing accumulators
Repository: flink Updated Branches: refs/heads/release-0.10 2d7f0b1af -> dfeee2372 [hotfix] Use AccumulatorSnapshot's class loader for deserializing accumulators This closes #1511 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/dfeee237 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/dfeee237 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/dfeee237 Branch: refs/heads/release-0.10 Commit: dfeee23721352544f02e3ec41234e05d95d56158 Parents: 2d7f0b1 Author: Prez CannadyAuthored: Thu Jan 14 18:58:41 2016 -0500 Committer: Stephan Ewen Committed: Fri Jan 15 12:22:36 2016 +0100 -- .../org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/dfeee237/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java index b813153..d0f4bad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java @@ -71,7 +71,7 @@ public class AccumulatorSnapshot implements Serializable { * @return the serialized map */ public Map deserializeFlinkAccumulators() throws IOException, ClassNotFoundException { - return flinkAccumulators.deserializeValue(ClassLoader.getSystemClassLoader()); + return flinkAccumulators.deserializeValue(getClass().getClassLoader()); } /**
flink git commit: [hotfix] Use AccumulatorSnapshot's class loader for deserializing accumulators
Repository: flink Updated Branches: refs/heads/master 2c556f74e -> cfcb5d7ba [hotfix] Use AccumulatorSnapshot's class loader for deserializing accumulators This closes #1511 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/cfcb5d7b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/cfcb5d7b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/cfcb5d7b Branch: refs/heads/master Commit: cfcb5d7ba5c22295c0ae628f8d9a2447e2286985 Parents: 2c556f7 Author: Prez CannadyAuthored: Thu Jan 14 18:58:41 2016 -0500 Committer: Stephan Ewen Committed: Fri Jan 15 12:23:42 2016 +0100 -- .../org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/cfcb5d7b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java index b813153..d0f4bad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java @@ -71,7 +71,7 @@ public class AccumulatorSnapshot implements Serializable { * @return the serialized map */ public Map deserializeFlinkAccumulators() throws IOException, ClassNotFoundException { - return flinkAccumulators.deserializeValue(ClassLoader.getSystemClassLoader()); + return flinkAccumulators.deserializeValue(getClass().getClassLoader()); } /**
[07/12] flink git commit: [FLINK-3235] Remove Flink on Tez code
http://git-wip-us.apache.org/repos/asf/flink/blob/8b8dfc16/flink-contrib/flink-tez/src/main/java/org/apache/flink/tez/dag/TezDAGGenerator.java -- diff --git a/flink-contrib/flink-tez/src/main/java/org/apache/flink/tez/dag/TezDAGGenerator.java b/flink-contrib/flink-tez/src/main/java/org/apache/flink/tez/dag/TezDAGGenerator.java deleted file mode 100644 index 52f39be..000 --- a/flink-contrib/flink-tez/src/main/java/org/apache/flink/tez/dag/TezDAGGenerator.java +++ /dev/null @@ -1,460 +0,0 @@ -/* - * 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.tez.dag; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.InputSplit; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.tez.runtime.TezTaskConfig; -import org.apache.flink.util.Visitor; -import org.apache.tez.dag.api.DAG; -import org.apache.tez.dag.api.TezConfiguration; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - - -public class TezDAGGenerator implements Visitor { - - private static final Log LOG = LogFactory.getLog(TezDAGGenerator.class); - - private Mapvertices; // a map from optimizer nodes to Tez vertices - private List edges; - private final int defaultMaxFan; - private final TezConfiguration tezConf; - - private final float defaultSortSpillingThreshold; - - public TezDAGGenerator (TezConfiguration tezConf, Configuration config) { - this.defaultMaxFan = config.getInteger(ConfigConstants.DEFAULT_SPILLING_MAX_FAN_KEY, - ConfigConstants.DEFAULT_SPILLING_MAX_FAN); - this.defaultSortSpillingThreshold = config.getFloat(ConfigConstants.DEFAULT_SORT_SPILLING_THRESHOLD_KEY, - ConfigConstants.DEFAULT_SORT_SPILLING_THRESHOLD); - this.tezConf = tezConf; - } - - public DAG createDAG (OptimizedPlan program) throws Exception { - LOG.info ("Creating Tez DAG"); - this.vertices = new HashMap (); - this.edges = new ArrayList(); - program.accept(this); - - DAG dag = DAG.create(program.getJobName()); - for (FlinkVertex v : vertices.values()) { - dag.addVertex(v.createVertex(new TezConfiguration(tezConf))); - } - for (FlinkEdge e: edges) { - dag.addEdge(e.createEdge(new TezConfiguration(tezConf))); - } - - /* -* Temporarily throw an error until TEZ-1190 has been fixed or a workaround has been created -*/ - if (containsSelfJoins())
[11/12] flink git commit: [hotfix] Minor cleanup of warnings, comments, and code style in the Java API Utils
[hotfix] Minor cleanup of warnings, comments, and code style in the Java API Utils Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/93654413 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/93654413 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/93654413 Branch: refs/heads/master Commit: 9365441396efdcf852e9076bdb6ca0fcc841434c Parents: e9a5358 Author: Stephan EwenAuthored: Thu Jan 14 22:08:41 2016 +0100 Committer: Stephan Ewen Committed: Fri Jan 15 11:44:21 2016 +0100 -- .../java/org/apache/flink/api/java/Utils.java | 68 1 file changed, 41 insertions(+), 27 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/93654413/flink-java/src/main/java/org/apache/flink/api/java/Utils.java -- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java index cb10906..038b58c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java @@ -19,12 +19,14 @@ package org.apache.flink.api.java; import org.apache.commons.lang3.StringUtils; + import org.apache.flink.api.common.accumulators.SerializedListAccumulator; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.configuration.Configuration; import java.io.IOException; import java.lang.reflect.Field; @@ -32,7 +34,6 @@ import java.lang.reflect.Modifier; import java.util.List; import java.util.Random; -import org.apache.flink.configuration.Configuration; import static org.apache.flink.api.java.functions.FunctionAnnotation.SkipCodeAnalysis; /** @@ -63,19 +64,28 @@ public final class Utils { * * @param typeInfo {@link CompositeType} */ - public static void getContainedGenericTypes(CompositeType typeInfo, List target) { - for(int i = 0; i < typeInfo.getArity(); i++) { + public static void getContainedGenericTypes(CompositeType typeInfo, List target) { + for (int i = 0; i < typeInfo.getArity(); i++) { TypeInformation type = typeInfo.getTypeAt(i); - if(type instanceof CompositeType) { - getContainedGenericTypes((CompositeType) type, target); - } else if(type instanceof GenericTypeInfo) { - if(!target.contains(type)) { + if (type instanceof CompositeType) { + getContainedGenericTypes((CompositeType) type, target); + } else if (type instanceof GenericTypeInfo) { + if (!target.contains(type)) { target.add((GenericTypeInfo) type); } } } } + // + + /** +* Utility sink function that counts elements and writes the count into an accumulator, +* from which it can be retrieved by the client. This sink is used by the +* {@link DataSet#count()} function. +* +* @param Type of elements to count. +*/ @SkipCodeAnalysis public static class CountHelper extends RichOutputFormat { @@ -90,24 +100,29 @@ public final class Utils { } @Override - public void configure(Configuration parameters) { - } + public void configure(Configuration parameters) {} @Override - public void open(int taskNumber, int numTasks) throws IOException { - } + public void open(int taskNumber, int numTasks) {} @Override - public void writeRecord(T record) throws IOException { + public void writeRecord(T record) { counter++; } @Override - public void close() throws IOException { + public void close() { getRuntimeContext().getLongCounter(id).add(counter); } } + /** +* Utility sink function that collects elements into an accumulator, +* from which it
[12/12] flink git commit: [FLINK-3197] [core] Close InputStream in BinaryInputFormat#createStatistics reliably
[FLINK-3197] [core] Close InputStream in BinaryInputFormat#createStatistics reliably This closes #1494 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2c556f74 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2c556f74 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2c556f74 Branch: refs/heads/master Commit: 2c556f74e491a47e7c10e7b8cc03e5c65cf34d23 Parents: 9365441 Author: Ajay BhatAuthored: Fri Jan 8 12:31:02 2016 +0530 Committer: Stephan Ewen Committed: Fri Jan 15 11:44:21 2016 +0100 -- .../flink/api/common/io/BinaryInputFormat.java | 16 +--- 1 file changed, 9 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/2c556f74/flink-core/src/main/java/org/apache/flink/api/common/io/BinaryInputFormat.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/BinaryInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/BinaryInputFormat.java index 61e3a1a..e738d52 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/BinaryInputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/BinaryInputFormat.java @@ -192,7 +192,7 @@ public abstract class BinaryInputFormat extends FileInputFormat { /** * Fill in the statistics. The last modification time and the total input size are prefilled. -* +* * @param files *The files that are associated with this block input format. * @param stats @@ -213,11 +213,13 @@ public abstract class BinaryInputFormat extends FileInputFormat { continue; } - FSDataInputStream fdis = file.getPath().getFileSystem().open(file.getPath(), blockInfo.getInfoSize()); - fdis.seek(file.getLen() - blockInfo.getInfoSize()); - - blockInfo.read(new DataInputViewStreamWrapper(fdis)); - totalCount += blockInfo.getAccumulatedRecordCount(); + FileSystem fs = file.getPath().getFileSystem(); + try (FSDataInputStream fdis = fs.open(file.getPath(), blockInfo.getInfoSize())) { + fdis.seek(file.getLen() - blockInfo.getInfoSize()); + + blockInfo.read(new DataInputViewStreamWrapper(fdis)); + totalCount += blockInfo.getAccumulatedRecordCount(); + } } final float avgWidth = totalCount == 0 ? 0 : ((float) stats.getTotalInputSize() / totalCount); @@ -270,7 +272,7 @@ public abstract class BinaryInputFormat extends FileInputFormat { if (this.reachedEnd()) { return null; } - + record = this.deserialize(record, this.dataInputStream); this.readRecords++; return record;
[02/12] flink git commit: [hotfix] Remove outdated README for the quickstart projects.
[hotfix] Remove outdated README for the quickstart projects. The README was describing non-existing bash scripts removed several versions ago. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/16cf8b13 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/16cf8b13 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/16cf8b13 Branch: refs/heads/master Commit: 16cf8b1355fda5cc47634820a88249d61a768c60 Parents: 9915b33 Author: Stephan EwenAuthored: Thu Jan 14 21:38:58 2016 +0100 Committer: Stephan Ewen Committed: Fri Jan 15 11:44:20 2016 +0100 -- flink-quickstart/README.md | 31 --- 1 file changed, 31 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/16cf8b13/flink-quickstart/README.md -- diff --git a/flink-quickstart/README.md b/flink-quickstart/README.md deleted file mode 100644 index e81cd57..000 --- a/flink-quickstart/README.md +++ /dev/null @@ -1,31 +0,0 @@ -flink-quickstart -=== - -Two simple quickstart maven archetypes for Flink. - -When you import the scala project into eclipse you will also need the following plugins: - -Eclipse 4.x: - * scala-ide: http://download.scala-ide.org/sdk/e38/scala210/stable/site - * m2eclipse-scala: http://alchim31.free.fr/m2e-scala/update-site - * build-helper-maven-plugin: https://repository.sonatype.org/content/repositories/forge-sites/m2e-extras/0.15.0/N/0.15.0.201206251206/ - -Eclipse 3.7: - * scala-ide: http://download.scala-ide.org/sdk/e37/scala210/stable/site - * m2eclipse-scala: http://alchim31.free.fr/m2e-scala/update-site - * build-helper-maven-plugin: https://repository.sonatype.org/content/repositories/forge-sites/m2e-extras/0.14.0/N/0.14.0.201109282148/ - - -# Repository Organization - -The quickstart bash scripts do not necessarily point to the most recent version in the code. Since the archetypes are versioned, the quickstarts usually differ by pointing to a specific version. - -The `quickstart.sh` script always points to the current stable release (v0.4, v0.5) -`-SNAPSHOT` points to the current snapshot version. - - -(Use `-DarchetypeCatalog=local` for local testing during archetype development) - -# Java 8 with Lambda Expressions - -If you are planning to use Java 8 and want to use Lambda Expression, please open the generated "pom.xml" file and modify/uncomment the mentioned lines.
[10/12] flink git commit: [FLINK-3219] [java scala] Implement DataSet.count and DataSet.collect using a single operator
[FLINK-3219] [java scala] Implement DataSet.count and DataSet.collect using a single operator Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/e9a53587 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/e9a53587 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/e9a53587 Branch: refs/heads/master Commit: e9a535877906bd75df3e633e3c5dad556b9c925d Parents: 0937be0 Author: Greg HoganAuthored: Mon Jan 11 17:04:31 2016 -0500 Committer: Stephan Ewen Committed: Fri Jan 15 11:44:21 2016 +0100 -- .../java/org/apache/flink/api/java/DataSet.java | 7 ++--- .../java/org/apache/flink/api/java/Utils.java | 33 +--- .../org/apache/flink/api/scala/DataSet.scala| 7 ++--- .../jsonplan/JsonJobGraphGenerationTest.java| 2 +- 4 files changed, 28 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/e9a53587/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java -- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index c5a636c..be84032 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -46,7 +46,6 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.SelectByMaxFunction; import org.apache.flink.api.java.functions.SelectByMinFunction; import org.apache.flink.api.java.io.CsvOutputFormat; -import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.api.java.io.PrintingOutputFormat; import org.apache.flink.api.java.io.TextOutputFormat; import org.apache.flink.api.java.io.TextOutputFormat.TextFormatter; @@ -387,8 +386,7 @@ public abstract class DataSet { public long count() throws Exception { final String id = new AbstractID().toString(); - flatMap(new Utils.CountHelper(id)).name("count()") - .output(new DiscardingOutputFormat()).name("count() sink"); + output(new Utils.CountHelper(id)).name("count()"); JobExecutionResult res = getExecutionEnvironment().execute(); return res. getAccumulatorResult(id); @@ -405,8 +403,7 @@ public abstract class DataSet { final String id = new AbstractID().toString(); final TypeSerializer serializer = getType().createSerializer(getExecutionEnvironment().getConfig()); - this.flatMap(new Utils.CollectHelper<>(id, serializer)).name("collect()") - .output(new DiscardingOutputFormat()).name("collect() sink"); + this.output(new Utils.CollectHelper<>(id, serializer)).name("collect()"); JobExecutionResult res = getExecutionEnvironment().execute(); ArrayList accResult = res.getAccumulatorResult(id); http://git-wip-us.apache.org/repos/asf/flink/blob/e9a53587/flink-java/src/main/java/org/apache/flink/api/java/Utils.java -- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java index 665f35f..cb10906 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java @@ -20,20 +20,19 @@ package org.apache.flink.api.java; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.accumulators.SerializedListAccumulator; +import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.util.List; import java.util.Random; -import org.apache.flink.api.common.functions.RichFlatMapFunction; - import org.apache.flink.configuration.Configuration; -import org.apache.flink.util.Collector; import static org.apache.flink.api.java.functions.FunctionAnnotation.SkipCodeAnalysis; /** @@ -78,7 +77,7 @@ public final class Utils { } @SkipCodeAnalysis - public static class CountHelper extends RichFlatMapFunction { + public static class CountHelper extends RichOutputFormat { private static final long serialVersionUID = 1L; @@ -91,18 +90,26 @@
[06/12] flink git commit: [FLINK-3235] Remove Flink on Tez code
http://git-wip-us.apache.org/repos/asf/flink/blob/8b8dfc16/flink-contrib/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java -- diff --git a/flink-contrib/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java b/flink-contrib/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java deleted file mode 100644 index 89e4642..000 --- a/flink-contrib/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java +++ /dev/null @@ -1,578 +0,0 @@ -/* - * 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.tez.runtime; - -import com.google.common.base.Preconditions; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.api.common.functions.util.RuntimeUDFContext; -import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.common.typeutils.TypeComparatorFactory; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.operators.Driver; -import org.apache.flink.runtime.operators.TaskContext; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.sort.CombiningUnilateralSortMerger; -import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; -import org.apache.flink.runtime.operators.util.CloseableInputProvider; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; -import org.apache.flink.tez.runtime.input.TezReaderIterator; -import org.apache.flink.tez.runtime.output.TezChannelSelector; -import org.apache.flink.tez.runtime.output.TezOutputEmitter; -import org.apache.flink.tez.runtime.output.TezOutputCollector; -import org.apache.flink.tez.util.DummyInvokable; -import org.apache.flink.util.Collector; -import org.apache.flink.util.InstantiationUtil; -import org.apache.flink.util.MutableObjectIterator; - -import org.apache.tez.runtime.library.api.KeyValueReader; -import org.apache.tez.runtime.library.api.KeyValueWriter; - -import java.io.IOException; -import java.net.URLClassLoader; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - - -public class TezTask implements TaskContext{ - - protected static final Log LOG = LogFactory.getLog(TezTask.class); - - DummyInvokable invokable = new DummyInvokable(); - - /** -* The driver that invokes the user code (the stub implementation). The central driver in this task -* (further drivers may be chained behind this driver). -*/ - protected volatile Driverdriver; - - /** -* The instantiated user code of this task's main operator (driver). May be null if the operator has no udf. -*/ - protected S stub; - - /** -* The udf's runtime context. -*/ - protected RuntimeUDFContext runtimeUdfContext; - - /** -* The collector that forwards the user code's results. May forward to a channel or to chained drivers within -* this task. -*/ - protected Collector output; - - /** -* The inputs reader, wrapped in an iterator. Prior to the local strategies, etc... -*/ - protected MutableObjectIterator[] inputIterators; - - /** -* The local strategies that are applied on the inputs. -*/ - protected volatile CloseableInputProvider[]
[04/12] flink git commit: [hotfix] Remove unnecessary dependencies in flink-runtime.
[hotfix] Remove unnecessary dependencies in flink-runtime. - Removes Apache Commons HTTP Client - Removes Apache Commons IO - Removes Jettison JSONparser and consolidates tests to use Jackson instead (which is included anyways) Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/388c280f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/388c280f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/388c280f Branch: refs/heads/master Commit: 388c280fcd8157f5ebc06a9062a0d06a3d2e92cb Parents: a65cd8d Author: Stephan EwenAuthored: Wed Jan 6 10:22:00 2016 +0100 Committer: Stephan Ewen Committed: Fri Jan 15 11:44:20 2016 +0100 -- .../main/resources/archetype-resources/pom.xml | 11 --- .../main/resources/archetype-resources/pom.xml | 13 --- flink-runtime/pom.xml | 17 .../runtime/webmonitor/WebMonitorUtils.java | 37 + .../flink/test/web/WebFrontendITCase.java | 85 .../flink/yarn/YARNSessionFIFOITCase.java | 29 --- tools/maven/checkstyle.xml | 6 ++ 7 files changed, 99 insertions(+), 99 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/388c280f/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml -- diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 1610a3a..4baf668 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -139,13 +139,6 @@ under the License. com.typesafe.akka:akka-slf4j_* io.netty:netty-all io.netty:netty - org.eclipse.jetty:jetty-server - org.eclipse.jetty:jetty-continuation - org.eclipse.jetty:jetty-http - org.eclipse.jetty:jetty-io - org.eclipse.jetty:jetty-util - org.eclipse.jetty:jetty-security - org.eclipse.jetty:jetty-servlet commons-fileupload:commons-fileupload org.apache.avro:avro commons-collections:commons-collections @@ -175,18 +168,14 @@ under the License. org.apache.commons:commons-math org.apache.sling:org.apache.sling.commons.json commons-logging:commons-logging - org.apache.httpcomponents:httpclient - org.apache.httpcomponents:httpcore commons-codec:commons-codec com.fasterxml.jackson.core:jackson-core com.fasterxml.jackson.core:jackson-databind com.fasterxml.jackson.core:jackson-annotations - org.codehaus.jettison:jettison stax:stax-api com.typesafe:config org.uncommons.maths:uncommons-maths com.github.scopt:scopt_* -
[08/12] flink git commit: [FLINK-3235] Remove Flink on Tez code
[FLINK-3235] Remove Flink on Tez code Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8b8dfc16 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8b8dfc16 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8b8dfc16 Branch: refs/heads/master Commit: 8b8dfc1671f21a2e381376e78184c7b46adf26b0 Parents: 16cf8b1 Author: Stephan EwenAuthored: Thu Jan 14 21:45:32 2016 +0100 Committer: Stephan Ewen Committed: Fri Jan 15 11:44:20 2016 +0100 -- .travis.yml | 2 +- docs/_includes/navbar.html | 1 - docs/index.md | 1 - docs/internals/general_arch.md | 1 - docs/setup/flink_on_tez.md | 291 -- flink-contrib/flink-tez/pom.xml | 224 --- .../flink-tez/src/assembly/flink-fat-jar.xml| 42 -- .../flink/tez/client/LocalTezEnvironment.java | 76 --- .../flink/tez/client/RemoteTezEnvironment.java | 83 --- .../apache/flink/tez/client/TezExecutor.java| 219 --- .../flink/tez/client/TezExecutorTool.java | 80 --- .../flink/tez/dag/FlinkBroadcastEdge.java | 70 --- .../flink/tez/dag/FlinkDataSinkVertex.java | 61 -- .../flink/tez/dag/FlinkDataSourceVertex.java| 82 --- .../org/apache/flink/tez/dag/FlinkEdge.java | 45 -- .../apache/flink/tez/dag/FlinkForwardEdge.java | 71 --- .../flink/tez/dag/FlinkPartitionEdge.java | 71 --- .../flink/tez/dag/FlinkProcessorVertex.java | 61 -- .../apache/flink/tez/dag/FlinkUnionVertex.java | 61 -- .../org/apache/flink/tez/dag/FlinkVertex.java | 114 .../apache/flink/tez/dag/TezDAGGenerator.java | 460 --- .../tez/examples/ConnectedComponentsStep.java | 203 --- .../flink/tez/examples/ExampleDriver.java | 119 .../flink/tez/examples/PageRankBasicStep.java | 241 .../apache/flink/tez/examples/TPCHQuery3.java | 224 --- .../examples/TransitiveClosureNaiveStep.java| 135 - .../apache/flink/tez/examples/WordCount.java| 129 - .../flink/tez/runtime/DataSinkProcessor.java| 228 .../flink/tez/runtime/DataSourceProcessor.java | 190 -- .../flink/tez/runtime/RegularProcessor.java | 138 - .../tez/runtime/TezRuntimeEnvironment.java | 44 -- .../org/apache/flink/tez/runtime/TezTask.java | 578 --- .../apache/flink/tez/runtime/TezTaskConfig.java | 163 -- .../flink/tez/runtime/UnionProcessor.java | 106 .../flink/tez/runtime/input/FlinkInput.java | 139 - .../runtime/input/FlinkInputSplitGenerator.java | 94 --- .../tez/runtime/input/TezReaderIterator.java| 66 --- .../tez/runtime/output/SimplePartitioner.java | 35 -- .../tez/runtime/output/TezChannelSelector.java | 36 -- .../tez/runtime/output/TezOutputCollector.java | 72 --- .../tez/runtime/output/TezOutputEmitter.java| 190 -- .../apache/flink/tez/util/DummyInvokable.java | 51 -- .../apache/flink/tez/util/EncodingUtils.java| 64 -- .../flink/tez/util/FlinkSerialization.java | 310 -- .../src/main/resources/log4j.properties | 30 - .../tez/test/ConnectedComponentsStepITCase.java | 83 --- .../flink/tez/test/PageRankBasicStepITCase.java | 54 -- .../flink/tez/test/TezProgramTestBase.java | 108 .../flink/tez/test/WebLogAnalysisITCase.java| 48 -- .../apache/flink/tez/test/WordCountITCase.java | 47 -- .../src/test/resources/log4j-test.properties| 30 - .../src/test/resources/logback-test.xml | 37 -- flink-contrib/pom.xml | 11 - flink-quickstart/flink-tez-quickstart/pom.xml | 37 -- .../src/main/java/Dummy.java| 28 - .../META-INF/maven/archetype-metadata.xml | 36 -- .../main/resources/archetype-resources/pom.xml | 186 -- .../src/assembly/flink-fat-jar.xml | 40 -- .../src/main/java/Driver.java | 113 .../src/main/java/LocalJob.java | 72 --- .../src/main/java/LocalWordCount.java | 96 --- .../src/main/java/YarnJob.java | 75 --- .../src/main/java/YarnWordCount.java| 124 .../projects/testArtifact/archetype.properties | 21 - .../resources/projects/testArtifact/goal.txt| 1 - flink-quickstart/pom.xml| 10 - pom.xml | 9 - 67 files changed, 1 insertion(+), 6966 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/8b8dfc16/.travis.yml -- diff --git a/.travis.yml b/.travis.yml index de27987..50b5fab 100644 --- a/.travis.yml +++
[03/12] flink git commit: [hotfix] Remove docs for the now removed web client.
[hotfix] Remove docs for the now removed web client. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9915b339 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9915b339 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9915b339 Branch: refs/heads/master Commit: 9915b339b77ad0a8b6bf61cf2df87877204201f5 Parents: 0ae46b5 Author: Stephan EwenAuthored: Thu Jan 14 20:37:36 2016 +0100 Committer: Stephan Ewen Committed: Fri Jan 15 11:44:20 2016 +0100 -- docs/_includes/navbar.html | 1 - docs/apis/web_client.md| 102 2 files changed, 103 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/9915b339/docs/_includes/navbar.html -- diff --git a/docs/_includes/navbar.html b/docs/_includes/navbar.html index 91ba62a..b5a04d9 100644 --- a/docs/_includes/navbar.html +++ b/docs/_includes/navbar.html @@ -92,7 +92,6 @@ under the License. Local Execution Cluster Execution Command Line Interface -Web Client Iterations (DataSet API) Java 8 Hadoop Compatibility Beta http://git-wip-us.apache.org/repos/asf/flink/blob/9915b339/docs/apis/web_client.md -- diff --git a/docs/apis/web_client.md b/docs/apis/web_client.md deleted file mode 100644 index 4749d90..000 --- a/docs/apis/web_client.md +++ /dev/null @@ -1,102 +0,0 @@ -title: "Web Client" - - -Flink provides a web interface to upload jobs, inspect their execution plans, and execute them. The interface is a great tool to showcase programs, debug execution plans, or demonstrate the system as a whole. - -* This will be replaced by the TOC -{:toc} - -## Starting, Stopping, and Configuring the Web Interface - -Start the web interface by executing: - -./bin/start-webclient.sh - -and stop it by calling: - -./bin/stop-webclient.sh - -The web interface runs on port 8080 by default. To specify a custom port set the ```webclient.port``` property in the *./conf/flink.yaml* configuration file. Jobs are submitted to the JobManager specified by ```jobmanager.rpc.address``` and ```jobmanager.rpc.port```. Please consult the [configuration]({{ site.baseurl }}/setup/config.html#webclient) page for details and further configuration options. - -## Using the Web Interface - -The web interface provides two views: - -1. The **job view** to upload, preview, and submit Flink programs. -2. The **plan view** to analyze the optimized execution plans of Flink programs. - -### Job View - - - - The interface starts serving the job view. - - You can upload a Flink program as a jar file. - - To execute an uploaded program: - - select it from the job list on the left, - enter (optional) execution options in the "Flink Options" field (bottom left), - enter (optional) program arguments in the "Program Arguments" field (bottom left), and - click on the "Run Job" button (bottom right). - - If the âShow optimizer planâ option is enabled (default), the plan view is display next, otherwise the job is directly submitted to the JobManager for execution. - - - - - - - - -The web interface can also handle multiple Flink jobs within a single jar file. To use this feature, package all required class files of all jobs into a single jar and specify the entry classes for each job as comma-separated-values in *program-class* argument within the jar's manifest file. The job view displays each entry class and you can pick any of them to preview the plan and/or submit the job to the JobManager. In case the jar's manifest file does not specify any entry class, you can specify it in the options field as: - -``` --c -``` - -It is also possible to set the default parallelism for the execution in options field as: - -``` --p -``` - -Furthermore, for each entry class implementing ```ProgramDescription``` interface, the provided description is shown as tooltip for the job (see {% gh_link flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCountMeta.java "WordCountMeta example" %}). - -### Plan View - - - - The plan view shows the optimized execution plan of the submitted program in the upper half of the page. The bottom part of the page displays detailed information about the currently selected plan
[05/12] flink git commit: [FLINK-3235] Remove Flink on Tez code
http://git-wip-us.apache.org/repos/asf/flink/blob/8b8dfc16/flink-contrib/flink-tez/src/test/resources/log4j-test.properties -- diff --git a/flink-contrib/flink-tez/src/test/resources/log4j-test.properties b/flink-contrib/flink-tez/src/test/resources/log4j-test.properties deleted file mode 100644 index 0845c81..000 --- a/flink-contrib/flink-tez/src/test/resources/log4j-test.properties +++ /dev/null @@ -1,30 +0,0 @@ - -# 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. - - -# Set root logger level to OFF to not flood build logs -# set manually to INFO for debugging purposes -log4j.rootLogger=INFO, testlogger - -# A1 is set to be a ConsoleAppender. -log4j.appender.testlogger=org.apache.log4j.ConsoleAppender -log4j.appender.testlogger.target = System.err -log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout -log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n - -# suppress the irrelevant (wrong) warnings from the netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger \ No newline at end of file http://git-wip-us.apache.org/repos/asf/flink/blob/8b8dfc16/flink-contrib/flink-tez/src/test/resources/logback-test.xml -- diff --git a/flink-contrib/flink-tez/src/test/resources/logback-test.xml b/flink-contrib/flink-tez/src/test/resources/logback-test.xml deleted file mode 100644 index 48e4374..000 --- a/flink-contrib/flink-tez/src/test/resources/logback-test.xml +++ /dev/null @@ -1,37 +0,0 @@ - - - - - -%d{HH:mm:ss.SSS} %-5level [%thread] %logger{60} - %msg%n - - - - - - - - - - - - - - - \ No newline at end of file http://git-wip-us.apache.org/repos/asf/flink/blob/8b8dfc16/flink-contrib/pom.xml -- diff --git a/flink-contrib/pom.xml b/flink-contrib/pom.xml index 5901621..78b0d76 100644 --- a/flink-contrib/pom.xml +++ b/flink-contrib/pom.xml @@ -44,15 +44,4 @@ under the License. flink-operator-stats flink-connector-wikiedits - - - - - include-tez - - flink-tez - - - - http://git-wip-us.apache.org/repos/asf/flink/blob/8b8dfc16/flink-quickstart/flink-tez-quickstart/pom.xml -- diff --git a/flink-quickstart/flink-tez-quickstart/pom.xml b/flink-quickstart/flink-tez-quickstart/pom.xml deleted file mode 100644 index 495fa01..000 --- a/flink-quickstart/flink-tez-quickstart/pom.xml +++ /dev/null @@ -1,37 +0,0 @@ - -http://maven.apache.org/POM/4.0.0; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; - xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd;> -4.0.0 - - -UTF-8 - - - -org.apache.flink -flink-quickstart -1.0-SNAPSHOT -.. - - -flink-tez-quickstart -maven-archetype - - http://git-wip-us.apache.org/repos/asf/flink/blob/8b8dfc16/flink-quickstart/flink-tez-quickstart/src/main/java/Dummy.java -- diff --git a/flink-quickstart/flink-tez-quickstart/src/main/java/Dummy.java b/flink-quickstart/flink-tez-quickstart/src/main/java/Dummy.java deleted file mode 100644 index c7749ff..000 --- a/flink-quickstart/flink-tez-quickstart/src/main/java/Dummy.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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
[01/12] flink git commit: [FLINK-3232] [runtime] Add option to eagerly deploy channels
Repository: flink Updated Branches: refs/heads/master a65cd8db1 -> 2c556f74e [FLINK-3232] [runtime] Add option to eagerly deploy channels Adds a flag to the ExecutionGraph's IntermediateResult class indicating whether the result consumers should be deployed eagerly. If true, the consumers are deployed as soon as the partition is registered at the ResultPartitionManager of the task manager. In practice, the deployment boils down to updating unknown input channels of the consumers (because the actual tasks are actually deployed all at once). This behaviour is configured in the JobGraph generator and only activated for streaming programs (StreamingJobGraphGenerator). It only makes sense for pipelined results. The motivation is to get down the latency of the first records passing a pipeline. The initial update of the input channels causes a higher latency. You can see this effect in the StreamingScalabilityAndLatency class (manual test). At the moment, this results in duplicate Akka messages when the first record is produced (the message travels from the task to the job manager and from the job manager to task manager, which then will be ignored at the InputGate). This closes #1503 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/0937be0a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0937be0a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0937be0a Branch: refs/heads/master Commit: 0937be0a94eae0b47f0a5f0206e62a98bcbc8432 Parents: 8b8dfc1 Author: Ufuk CelebiAuthored: Wed Jan 13 18:45:35 2016 +0100 Committer: Stephan Ewen Committed: Fri Jan 15 11:44:20 2016 +0100 -- .../ResultPartitionDeploymentDescriptor.java| 25 +++- .../executiongraph/ExecutionJobVertex.java | 6 +- .../executiongraph/IntermediateResult.java | 11 +- .../runtime/io/network/NetworkEnvironment.java | 13 +++ .../io/network/partition/ResultPartition.java | 20 .../runtime/jobgraph/IntermediateDataSet.java | 31 + .../flink/runtime/jobgraph/JobVertex.java | 13 ++- .../apache/flink/runtime/taskmanager/Task.java | 1 + ...ResultPartitionDeploymentDescriptorTest.java | 60 ++ .../io/network/NetworkEnvironmentTest.java | 113 ++- .../consumer/LocalInputChannelTest.java | 1 + .../runtime/jobgraph/JobTaskVertexTest.java | 33 +- .../runtime/taskmanager/TaskManagerTest.java| 4 +- .../api/graph/StreamingJobGraphGenerator.java | 13 ++- 14 files changed, 327 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/0937be0a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java index 2b0bbc1..eadcd1c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java @@ -48,11 +48,20 @@ public class ResultPartitionDeploymentDescriptor implements Serializable { /** The number of subpartitions. */ private final int numberOfSubpartitions; + /** +* Flag indicating whether to eagerly deploy consumers. +* +* If true, the consumers are deployed as soon as the +* runtime result is registered at the result manager of the task manager. +*/ + private final boolean eagerlyDeployConsumers; + public ResultPartitionDeploymentDescriptor( IntermediateDataSetID resultId, IntermediateResultPartitionID partitionId, ResultPartitionType partitionType, - int numberOfSubpartitions) { + int numberOfSubpartitions, + boolean eagerlyDeployConsumers) { this.resultId = checkNotNull(resultId); this.partitionId = checkNotNull(partitionId); @@ -60,6 +69,7 @@ public class ResultPartitionDeploymentDescriptor implements Serializable { checkArgument(numberOfSubpartitions >= 1); this.numberOfSubpartitions = numberOfSubpartitions; + this.eagerlyDeployConsumers = eagerlyDeployConsumers; } public IntermediateDataSetID getResultId() { @@ -78,6 +88,16 @@ public class ResultPartitionDeploymentDescriptor implements Serializable { return numberOfSubpartitions;
[09/12] flink git commit: [FLINK-3236] [runtime] Flink user code classloader as parent classloader from Flink core classes
[FLINK-3236] [runtime] Flink user code classloader as parent classloader from Flink core classes Originally, the user code classloader delegates to the system classloader as parent. That works in Flink standalone settings, but not when the Flink core classes themselves are not loaded with the system classloader (certain embedded setups). This patch uses as parent the classloader that was used to load the Flink core classes, specificly "org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager.FlinkUserCodeClassLoader.class.getClassLoader()". This closes #1506 This closes #1507 This closes #1508 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/0ae46b59 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0ae46b59 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0ae46b59 Branch: refs/heads/master Commit: 0ae46b596949808f56c40bd7a68f478bc10206ab Parents: 388c280 Author: Prez CannadyAuthored: Thu Jan 14 10:15:12 2016 -0500 Committer: Stephan Ewen Committed: Fri Jan 15 11:44:20 2016 +0100 -- .../runtime/execution/librarycache/BlobLibraryCacheManager.java| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/0ae46b59/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java index 97ec93a..98caac4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java @@ -326,7 +326,7 @@ public final class BlobLibraryCacheManager extends TimerTask implements LibraryC private static class FlinkUserCodeClassLoader extends URLClassLoader { public FlinkUserCodeClassLoader(URL[] urls) { - super(urls); + super(urls, FlinkUserCodeClassLoader.class.getClassLoader()); } } }
[2/2] flink git commit: [FLINK-2716] [gelly] [apis] New checksum method on DataSet and Graph
[FLINK-2716] [gelly] [apis] New checksum method on DataSet and Graph This closes #1462 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/a8be52a3 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/a8be52a3 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/a8be52a3 Branch: refs/heads/master Commit: a8be52a3309c5e68fe865a421fb60fdb3cc56d16 Parents: 98fad04 Author: Greg HoganAuthored: Tue Dec 15 14:25:54 2015 -0500 Committer: Stephan Ewen Committed: Fri Jan 15 18:58:57 2016 +0100 -- .../java/org/apache/flink/api/java/Utils.java | 93 .../flink/api/java/utils/DataSetUtils.java | 21 + .../flink/graph/scala/utils/package.scala | 50 +++ .../scala/test/util/GraphUtilsITCase.scala | 46 ++ .../apache/flink/graph/utils/GraphUtils.java| 38 .../flink/graph/test/util/GraphUtilsITCase.java | 58 .../apache/flink/api/scala/utils/package.scala | 21 + .../flink/test/util/DataSetUtilsITCase.java | 13 +++ .../api/scala/util/DataSetUtilsITCase.scala | 12 +++ 9 files changed, 352 insertions(+) -- http://git-wip-us.apache.org/repos/asf/flink/blob/a8be52a3/flink-java/src/main/java/org/apache/flink/api/java/Utils.java -- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java index 038b58c..2edc533 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java @@ -20,7 +20,9 @@ package org.apache.flink.api.java; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.SerializedListAccumulator; +import org.apache.flink.api.common.accumulators.SimpleAccumulator; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; @@ -158,6 +160,97 @@ public final class Utils { } } + public static class ChecksumHashCode implements SimpleAccumulator { + + private static final long serialVersionUID = 1L; + + private long count; + private long checksum; + + public ChecksumHashCode() {} + + public ChecksumHashCode(long count, long checksum) { + this.count = count; + this.checksum = checksum; + } + + public long getCount() { + return count; + } + + public long getChecksum() { + return checksum; + } + + @Override + public void add(ChecksumHashCode value) { + this.count += value.count; + this.checksum += value.checksum; + } + + @Override + public ChecksumHashCode getLocalValue() { + return this; + } + + @Override + public void resetLocal() { + this.count = 0; + this.checksum = 0; + } + + @Override + public void merge(Accumulator other) { + this.add(other.getLocalValue()); + } + + @Override + public ChecksumHashCode clone() { + return new ChecksumHashCode(count, checksum); + } + + @Override + public String toString() { + return "ChecksumHashCode " + this.checksum + ", count " + this.count; + } + } + + @SkipCodeAnalysis + public static class ChecksumHashCodeHelper extends RichOutputFormat { + + private static final long serialVersionUID = 1L; + + private final String id; + private long counter; + private long checksum; + + public ChecksumHashCodeHelper(String id) { + this.id = id; + this.counter = 0L; + this.checksum = 0L; + } + + @Override + public void configure(Configuration parameters) {} + + @Override + public void open(int taskNumber, int numTasks) {} + + @Override + public void writeRecord(T record) throws IOException { + counter++; +
[1/2] flink git commit: [FLINK-2671] [tests] Fix unstable StreamCheckpointNotifierITCase
Repository: flink Updated Branches: refs/heads/master 98fad04ec -> c0fd36bac [FLINK-2671] [tests] Fix unstable StreamCheckpointNotifierITCase Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c0fd36ba Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c0fd36ba Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c0fd36ba Branch: refs/heads/master Commit: c0fd36bac8955cff58a88e72d5bdb378f3cfdf93 Parents: a8be52a Author: Stephan EwenAuthored: Fri Jan 15 17:29:46 2016 +0100 Committer: Stephan Ewen Committed: Fri Jan 15 18:58:57 2016 +0100 -- .../api/functions/sink/DiscardingSink.java | 32 ++ .../StreamCheckpointNotifierITCase.java | 402 +++ 2 files changed, 276 insertions(+), 158 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/c0fd36ba/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/DiscardingSink.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/DiscardingSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/DiscardingSink.java new file mode 100644 index 000..3bbb14b --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/DiscardingSink.java @@ -0,0 +1,32 @@ +/* + * 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.streaming.api.functions.sink; + +/** + * A stream sink that ignores all elements. + * + * @param The type of elements received by the sink. + */ +public class DiscardingSink implements SinkFunction { + + private static final long serialVersionUID = 1L; + + @Override + public void invoke(T value) {} +} http://git-wip-us.apache.org/repos/asf/flink/blob/c0fd36ba/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java -- diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java index 08af93a..9de5794 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java @@ -22,31 +22,41 @@ import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.Collector; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashSet; import java.util.List; import
flink git commit: [hotfix] Exclude 'flink-annotations' from fat jars in quickstart poms.
Repository: flink Updated Branches: refs/heads/master 7d7d8b68e -> 153a67881 [hotfix] Exclude 'flink-annotations' from fat jars in quickstart poms. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/153a6788 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/153a6788 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/153a6788 Branch: refs/heads/master Commit: 153a67881fa97a78f96f50220bea45195df18940 Parents: 7d7d8b6 Author: Stephan EwenAuthored: Mon Jan 18 11:37:57 2016 +0100 Committer: Stephan Ewen Committed: Mon Jan 18 17:51:29 2016 +0100 -- .../src/main/resources/archetype-resources/pom.xml | 1 + .../src/main/resources/archetype-resources/pom.xml | 1 + 2 files changed, 2 insertions(+) -- http://git-wip-us.apache.org/repos/asf/flink/blob/153a6788/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml -- diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 4baf668..1ff3273 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -111,6 +111,7 @@ under the License. + org.apache.flink:flink-annotations org.apache.flink:flink-shaded-* org.apache.flink:flink-core org.apache.flink:flink-java http://git-wip-us.apache.org/repos/asf/flink/blob/153a6788/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml -- diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index 560a29f..c7ec411 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -115,6 +115,7 @@ under the License. + org.apache.flink:flink-annotations org.apache.flink:flink-shaded-* org.apache.flink:flink-core org.apache.flink:flink-java
[2/2] flink git commit: [hotfix] Clean up CliFrontend after removing web client
[hotfix] Clean up CliFrontend after removing web client Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/17fa6a9b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/17fa6a9b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/17fa6a9b Branch: refs/heads/master Commit: 17fa6a9bc965eb4dcd64123b7d1c66a75c077db6 Parents: c0fd36b Author: Stephan EwenAuthored: Fri Jan 15 18:36:13 2016 +0100 Committer: Stephan Ewen Committed: Sat Jan 16 15:46:56 2016 +0100 -- .../org/apache/flink/client/CliFrontend.java| 125 --- .../flink/yarn/FlinkYarnSessionCliTest.java | 10 +- .../apache/flink/yarn/FlinkYarnClientBase.java | 9 +- .../flink/yarn/ApplicationMasterBase.scala | 8 +- 4 files changed, 62 insertions(+), 90 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/17fa6a9b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java -- diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java index 7e1cef7..4b9bd06 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java @@ -19,13 +19,14 @@ package org.apache.flink.client; import akka.actor.ActorSystem; + import org.apache.commons.cli.CommandLine; + import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.accumulators.AccumulatorHelper; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.cli.CancelOptions; import org.apache.flink.client.cli.CliArgsException; import org.apache.flink.client.cli.CliFrontendParser; @@ -65,8 +66,10 @@ import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus; import org.apache.flink.util.StringUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import scala.Some; import scala.concurrent.Await; import scala.concurrent.Future; @@ -87,6 +90,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.Date; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -95,7 +99,6 @@ import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint; import static org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepointFailure; import static org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointFailure; -import static org.apache.flink.runtime.messages.JobManagerMessages.getRequestRunningJobsStatus; /** * Implementation of a simple command line frontend for executing programs. @@ -133,6 +136,7 @@ public class CliFrontend { private static final Logger LOG = LoggerFactory.getLogger(CliFrontend.class); + private final Configuration config; private final FiniteDuration askTimeout; @@ -143,12 +147,6 @@ public class CliFrontend { private AbstractFlinkYarnCluster yarnCluster; - static boolean webFrontend = false; - - private FlinkPlan optimizedPlan; - - private PackagedProgram packagedProgram; - /** * * @throws Exception Thrown if the configuration directory was not found, the configuration could not @@ -222,9 +220,9 @@ public class CliFrontend { // handle the YARN client's dynamic properties String dynamicPropertiesEncoded = yarnProperties.getProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING); - List > dynamicProperties = getDynamicProperties(dynamicPropertiesEncoded); - for (Tuple2 dynamicProperty : dynamicProperties) { - this.config.setString(dynamicProperty.f0, dynamicProperty.f1); + Map dynamicProperties = getDynamicProperties(dynamicPropertiesEncoded); + for (Map.Entry dynamicProperty : dynamicProperties.entrySet()) { + this.config.setString(dynamicProperty.getKey(), dynamicProperty.getValue()); } } @@ -408,42 +406,34 @@ public class CliFrontend { LOG.info("Creating program plan dump");
[1/2] flink git commit: [hotfix] Fix reference to batch Java examples in flink-java8
Repository: flink Updated Branches: refs/heads/master c0fd36bac -> adbeec2f9 [hotfix] Fix reference to batch Java examples in flink-java8 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/adbeec2f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/adbeec2f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/adbeec2f Branch: refs/heads/master Commit: adbeec2f912289779e4561cfafda27e429eb1051 Parents: 17fa6a9 Author: Stephan EwenAuthored: Fri Jan 15 22:13:46 2016 +0100 Committer: Stephan Ewen Committed: Sat Jan 16 15:46:56 2016 +0100 -- flink-java8/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/adbeec2f/flink-java8/pom.xml -- diff --git a/flink-java8/pom.xml b/flink-java8/pom.xml index 33ea929..b0f18c7 100644 --- a/flink-java8/pom.xml +++ b/flink-java8/pom.xml @@ -66,7 +66,7 @@ under the License. org.apache.flink - flink-java-examples + flink-examples-batch ${project.version}
flink git commit: [FLINK-3236] [runtime] Flink user code classloader as parent classloader from Flink core classes
Repository: flink Updated Branches: refs/heads/release-0.10 d09a83320 -> 2d7f0b1af [FLINK-3236] [runtime] Flink user code classloader as parent classloader from Flink core classes Originally, the user code classloader delegates to the system classloader as parent. That works in Flink standalone settings, but not when the Flink core classes themselves are not loaded with the system classloader (certain embedded setups). This patch uses as parent the classloader that was used to load the Flink core classes, specificly "org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager.FlinkUserCodeClassLoader.class.getClassLoader()". Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2d7f0b1a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2d7f0b1a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2d7f0b1a Branch: refs/heads/release-0.10 Commit: 2d7f0b1af7f490e6f341ed0b08aa561a67e457b5 Parents: d09a833 Author: Prez CannadyAuthored: Thu Jan 14 10:15:12 2016 -0500 Committer: Stephan Ewen Committed: Thu Jan 14 21:48:09 2016 +0100 -- .../runtime/execution/librarycache/BlobLibraryCacheManager.java| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/2d7f0b1a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java index 97ec93a..98caac4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java @@ -326,7 +326,7 @@ public final class BlobLibraryCacheManager extends TimerTask implements LibraryC private static class FlinkUserCodeClassLoader extends URLClassLoader { public FlinkUserCodeClassLoader(URL[] urls) { - super(urls); + super(urls, FlinkUserCodeClassLoader.class.getClassLoader()); } } }
flink git commit: [FLINK-3373] [build] Shade away Hadoop's HTTP Components dependency
Repository: flink Updated Branches: refs/heads/master aeee6efd4 -> 8ccd7544e [FLINK-3373] [build] Shade away Hadoop's HTTP Components dependency This closes #1615 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8ccd7544 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8ccd7544 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8ccd7544 Branch: refs/heads/master Commit: 8ccd7544edb25e82cc8a898809cc7c8bb7893620 Parents: aeee6ef Author: Stephan EwenAuthored: Tue Feb 9 21:18:43 2016 +0100 Committer: Stephan Ewen Committed: Wed Feb 10 15:01:22 2016 +0100 -- flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml | 15 +++ flink-shaded-hadoop/pom.xml | 13 + pom.xml | 12 3 files changed, 28 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/8ccd7544/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml -- diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml index b5839d9..5eb8043 100644 --- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml +++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml @@ -652,4 +652,19 @@ under the License. + + + + org.apache.httpcomponents + httpcore + 4.2.5 + + + + org.apache.httpcomponents + httpclient + 4.2.6 + + + http://git-wip-us.apache.org/repos/asf/flink/blob/8ccd7544/flink-shaded-hadoop/pom.xml -- diff --git a/flink-shaded-hadoop/pom.xml b/flink-shaded-hadoop/pom.xml index 7d54ef9..d5a8529 100644 --- a/flink-shaded-hadoop/pom.xml +++ b/flink-shaded-hadoop/pom.xml @@ -111,6 +111,11 @@ under the License. io.netty:netty:* org.apache.curator:* org.apache.hadoop:* + + + net.java.dev.jets3t:jets3t + org.apache.httpcomponents:* + commons-httpclient:commons-httpclient @@ -133,6 +138,14 @@ under the License. org.apache.curator org.apache.flink.hadoop.shaded.org.apache.curator + + org.apache.http + org.apache.flink.hadoop.shaded.org.apache.http + + + org.apache.commons.httpclient + org.apache.flink.hadoop.shaded.org.apache.commons.httpclient + http://git-wip-us.apache.org/repos/asf/flink/blob/8ccd7544/pom.xml -- diff --git a/pom.xml b/pom.xml index 0a64c4a..42ebf79 100644 --- a/pom.xml +++ b/pom.xml @@ -342,18 +342,6 @@ under the License. - org.apache.httpcomponents - httpcore - 4.2.5 - - - - org.apache.httpcomponents
flink git commit: [FLINK-3372] Setting custom YARN application name is ignored
Repository: flink Updated Branches: refs/heads/release-0.10 2cd06181f -> 11bfe6f3e [FLINK-3372] Setting custom YARN application name is ignored This closes #1607 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/11bfe6f3 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/11bfe6f3 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/11bfe6f3 Branch: refs/heads/release-0.10 Commit: 11bfe6f3e9956101e76fbb3b3b8dccee93d81eb3 Parents: 2cd0618 Author: Nick DimidukAuthored: Mon Feb 8 10:24:52 2016 -0800 Committer: Stephan Ewen Committed: Wed Feb 10 15:20:49 2016 +0100 -- .../src/main/java/org/apache/flink/client/CliFrontend.java | 3 +-- .../main/java/org/apache/flink/client/FlinkYarnSessionCli.java | 6 ++ 2 files changed, 7 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/11bfe6f3/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java -- diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java index 7c8d52e..d830361 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java @@ -813,11 +813,10 @@ public class CliFrontend { // user wants to run Flink in YARN cluster. CommandLine commandLine = options.getCommandLine(); - AbstractFlinkYarnClient flinkYarnClient = CliFrontendParser.getFlinkYarnSessionCli().createFlinkYarnClient(commandLine); + AbstractFlinkYarnClient flinkYarnClient = CliFrontendParser.getFlinkYarnSessionCli().createFlinkYarnClient(commandLine, programName); if (flinkYarnClient == null) { throw new RuntimeException("Unable to create Flink YARN Client. Check previous log messages"); } - flinkYarnClient.setName("Flink Application: " + programName); // the number of slots available from YARN: int yarnTmSlots = flinkYarnClient.getTaskManagerSlots(); http://git-wip-us.apache.org/repos/asf/flink/blob/11bfe6f3/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java -- diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java index a9a20ae..ff11c19 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java @@ -103,6 +103,10 @@ public class FlinkYarnSessionCli { } public AbstractFlinkYarnClient createFlinkYarnClient(CommandLine cmd) { + return createFlinkYarnClient(cmd, null); + } + + public AbstractFlinkYarnClient createFlinkYarnClient(CommandLine cmd, String programName) { AbstractFlinkYarnClient flinkYarnClient = getFlinkYarnClient(); if (flinkYarnClient == null) { @@ -225,6 +229,8 @@ public class FlinkYarnSessionCli { } if(cmd.hasOption(NAME.getOpt())) { flinkYarnClient.setName(cmd.getOptionValue(NAME.getOpt())); + } else if (programName != null && !programName.isEmpty()) { + flinkYarnClient.setName("Flink Application: " + programName); } return flinkYarnClient; }
[6/6] flink git commit: [FLINK-3260] [runtime] Enforce terminal state of Executions
[FLINK-3260] [runtime] Enforce terminal state of Executions This commit fixes the problem that Executions could leave their terminal state FINISHED to transition to FAILED. Such a transition will be propagated to the ExecutionGraph where it entails JobStatus changes. Since the Execution already reached a terminal state, it should not again affect the ExecutionGraph. This can lead to an inconsistent state in case of a restart where the old Executions get disassociated from the ExecutionGraph. This closes #1613 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6968a57a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6968a57a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6968a57a Branch: refs/heads/master Commit: 6968a57a1a31a11b33bacd2c94d6559bcabd6eb9 Parents: 48b7454 Author: Till RohrmannAuthored: Tue Feb 9 10:30:12 2016 +0100 Committer: Stephan Ewen Committed: Wed Feb 10 15:34:37 2016 +0100 -- .../flink/runtime/executiongraph/Execution.java | 14 +- .../ExecutionGraphRestartTest.java | 90 + .../runtime/testingUtils/TestingCluster.scala | 6 +- .../testingUtils/TestingTaskManagerLike.scala | 4 +- .../runtime/testingUtils/TestingUtils.scala | 133 ++- 5 files changed, 233 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6968a57a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index eb2e68c..db037bb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -435,7 +435,7 @@ public class Execution implements Serializable { return; } else if (current == CREATED || current == SCHEDULED) { - // from here, we can directly switch to cancelled, because the no task has been deployed + // from here, we can directly switch to cancelled, because no task has been deployed if (transitionState(current, CANCELED)) { // we skip the canceling state. set the timestamp, for a consistent appearance @@ -754,11 +754,10 @@ public class Execution implements Serializable { return false; } - if (current == CANCELED) { - // we are already aborting or are already aborted + if (current == CANCELED || current == FINISHED) { + // we are already aborting or are already aborted or we are already finished if (LOG.isDebugEnabled()) { - LOG.debug(String.format("Ignoring transition of vertex %s to %s while being %s", - getVertexWithAttempt(), FAILED, CANCELED)); + LOG.debug("Ignoring transition of vertex {} to {} while being {}.", getVertexWithAttempt(), FAILED, current); } return false; } @@ -928,6 +927,11 @@ public class Execution implements Serializable { } private boolean transitionState(ExecutionState currentState, ExecutionState targetState, Throwable error) { + // sanity check + if (currentState.isTerminal()) { + throw new IllegalStateException("Cannot leave terminal state " + currentState + " to transition to " + targetState + "."); + } + if (STATE_UPDATER.compareAndSet(this, currentState, targetState)) { markTimestamp(targetState); http://git-wip-us.apache.org/repos/asf/flink/blob/6968a57a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index 0c3af8f..47a48a2 100644 ---
[3/6] flink git commit: [hotfix] [tests] Ignore ZooKeeper logs in process tests
[hotfix] [tests] Ignore ZooKeeper logs in process tests Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3a643c07 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3a643c07 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3a643c07 Branch: refs/heads/master Commit: 3a643c07792c62142c1f8cda172d4f4c3442c9b3 Parents: b8f4025 Author: Ufuk CelebiAuthored: Tue Feb 9 11:01:39 2016 +0100 Committer: Stephan Ewen Committed: Wed Feb 10 15:27:16 2016 +0100 -- .../JobManagerSubmittedJobGraphsRecoveryITCase.java | 6 +- .../org/apache/flink/runtime/testutils/CommonTestUtils.java | 1 + .../AbstractJobManagerProcessFailureRecoveryITCase.java | 8 +--- .../org/apache/flink/test/recovery/ChaosMonkeyITCase.java| 4 .../test/recovery/JobManagerCheckpointRecoveryITCase.java| 8 5 files changed, 23 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/3a643c07/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java index 99f7bd7..59c7c39 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java @@ -343,6 +343,10 @@ public class JobManagerSubmittedJobGraphsRecoveryITCase extends TestLogger { assertEquals(2, jobSubmitSuccessMessages); } catch (Throwable t) { + // Print early (in some situations the process logs get too big + // for Travis and the root problem is not shown) + t.printStackTrace(); + // In case of an error, print the job manager process logs. if (jobManagerProcess[0] != null) { jobManagerProcess[0].printProcessLog(); @@ -352,7 +356,7 @@ public class JobManagerSubmittedJobGraphsRecoveryITCase extends TestLogger { jobManagerProcess[1].printProcessLog(); } - t.printStackTrace(); + throw t; } finally { if (jobManagerProcess[0] != null) { http://git-wip-us.apache.org/repos/asf/flink/blob/3a643c07/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java index 069b6af..bbb6a89 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java @@ -147,6 +147,7 @@ public class CommonTestUtils { writer.println("log4j.appender.console.layout=org.apache.log4j.PatternLayout"); writer.println("log4j.appender.console.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n"); writer.println("log4j.logger.org.eclipse.jetty.util.log=OFF"); + writer.println("log4j.logger.org.apache.zookeeper=OFF"); writer.flush(); writer.close(); http://git-wip-us.apache.org/repos/asf/flink/blob/3a643c07/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java -- diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java index 2f6b762..6122352 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java @@ -246,8 +246,10 @@ public abstract class AbstractJobManagerProcessFailureRecoveryITCase extends Tes fail("The program encountered a " + error.getClass().getSimpleName() + "
[1/6] flink git commit: [FLINK-3341] Make 'auto.offset.reset' compatible with Kafka 0.8 and 0.9
Repository: flink Updated Branches: refs/heads/master 8ccd7544e -> 6968a57a1 [FLINK-3341] Make 'auto.offset.reset' compatible with Kafka 0.8 and 0.9 This closes #1597 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9173825a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9173825a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9173825a Branch: refs/heads/master Commit: 9173825aa6a1525d72a78cda16cb4ae1e9b8a8e4 Parents: 8ccd754 Author: Robert MetzgerAuthored: Sat Feb 6 13:27:06 2016 +0100 Committer: Stephan Ewen Committed: Wed Feb 10 15:12:34 2016 +0100 -- .../connectors/kafka/FlinkKafkaConsumer08.java | 2 +- .../kafka/internals/LegacyFetcher.java | 3 ++- .../connectors/kafka/Kafka08ITCase.java | 22 ++-- .../kafka/KafkaTestEnvironmentImpl.java | 20 +++--- .../kafka/KafkaTestEnvironmentImpl.java | 22 +--- .../connectors/kafka/KafkaConsumerTestBase.java | 5 ++--- .../connectors/kafka/KafkaTestBase.java | 2 -- .../connectors/kafka/KafkaTestEnvironment.java | 3 --- .../flink/yarn/YARNSessionFIFOITCase.java | 1 + 9 files changed, 25 insertions(+), 55 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/9173825a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java -- diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java index bdea37f..1cdfffe 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java @@ -70,7 +70,7 @@ import static com.google.common.base.Preconditions.checkNotNull; * socket.timeout.ms * socket.receive.buffer.bytes * fetch.message.max.bytes - * auto.offset.reset with the values "latest", "earliest" (unlike 0.8.2 behavior) + * auto.offset.reset with the values "largest", "smallest" * fetch.wait.max.ms * * http://git-wip-us.apache.org/repos/asf/flink/blob/9173825a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java -- diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java index fe7f777..10f4c41 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java @@ -576,7 +576,8 @@ public class LegacyFetcher implements Fetcher { private static long getInvalidOffsetBehavior(Properties config) { long timeType; - if (config.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest").equals("latest")) { + String val = config.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "largest"); + if (val.equals("largest") || val.equals("latest")) { // largest is kafka 0.8, latest is kafka 0.9 timeType = OffsetRequest.LatestTime(); } else { timeType = OffsetRequest.EarliestTime(); http://git-wip-us.apache.org/repos/asf/flink/blob/9173825a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java -- diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java index 6a2fa27..a3e815e 100644 ---
[2/6] flink git commit: [FLINK-3350] [tests] Increase default test Akka ask and ZooKeeper timeouts
[FLINK-3350] [tests] Increase default test Akka ask and ZooKeeper timeouts Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b8f40251 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b8f40251 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b8f40251 Branch: refs/heads/master Commit: b8f40251c6c45379118254c21b0d553c2d3b8937 Parents: 9173825 Author: Ufuk CelebiAuthored: Mon Feb 8 14:24:43 2016 +0100 Committer: Stephan Ewen Committed: Wed Feb 10 15:26:43 2016 +0100 -- .../runtime/minicluster/FlinkMiniCluster.scala | 20 ++-- .../minicluster/LocalFlinkMiniCluster.scala | 2 ++ .../runtime/testutils/ZooKeeperTestUtils.java | 5 +++-- .../runtime/testingUtils/TestingCluster.scala | 2 ++ .../kafka/KafkaTestEnvironmentImpl.java | 7 --- .../kafka/KafkaTestEnvironmentImpl.java | 7 --- ...ctTaskManagerProcessFailureRecoveryTest.java | 3 +++ .../JobManagerCheckpointRecoveryITCase.java | 8 ++-- .../recovery/ProcessFailureCancelingITCase.java | 2 +- 9 files changed, 43 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/b8f40251/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala -- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala index 4cdda3f..0346d6d 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala @@ -42,7 +42,7 @@ import org.apache.flink.runtime.webmonitor.{WebMonitorUtils, WebMonitor} import org.slf4j.LoggerFactory -import scala.concurrent.duration.FiniteDuration +import scala.concurrent.duration.{Duration, FiniteDuration} import scala.concurrent._ import scala.concurrent.forkjoin.ForkJoinPool @@ -86,7 +86,7 @@ abstract class FlinkMiniCluster( implicit val executionContext = ExecutionContext.global - implicit val timeout = AkkaUtils.getTimeout(userConfiguration) + implicit val timeout = AkkaUtils.getTimeout(configuration) val recoveryMode = RecoveryMode.fromConfig(configuration) @@ -188,6 +188,22 @@ abstract class FlinkMiniCluster( AkkaUtils.getAkkaConfig(configuration, Some((hostname, resolvedPort))) } + /** +* Sets CI environment (Travis) specific config defaults. +*/ + def setDefaultCiConfig(config: Configuration) : Unit = { +// https://docs.travis-ci.com/user/environment-variables#Default-Environment-Variables +if (sys.env.contains("CI")) { + // Only set if nothing specified in config + if (config.getString(ConfigConstants.AKKA_ASK_TIMEOUT, null) == null) { +val duration = Duration(ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT) * 10 +config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, s"${duration.toSeconds}s") + +LOG.info(s"Akka ask timeout set to ${duration.toSeconds}s") + } +} + } + // -- // Start/Stop Methods // -- http://git-wip-us.apache.org/repos/asf/flink/blob/b8f40251/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala -- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala index 913aec0..c803429 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala @@ -48,6 +48,8 @@ class LocalFlinkMiniCluster( override def generateConfiguration(userConfiguration: Configuration): Configuration = { val config = getDefaultConfig +setDefaultCiConfig(config) + config.addAll(userConfiguration) setMemory(config) initializeIOFormatClasses(config) http://git-wip-us.apache.org/repos/asf/flink/blob/b8f40251/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java index
[4/6] flink git commit: [hotfix] [tests] Log retry rule failures on warn level
[hotfix] [tests] Log retry rule failures on warn level Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/756cbaff Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/756cbaff Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/756cbaff Branch: refs/heads/master Commit: 756cbafff1fd25f67268ca84b62c8a479156bf88 Parents: 3a643c0 Author: Ufuk CelebiAuthored: Tue Feb 9 11:25:37 2016 +0100 Committer: Stephan Ewen Committed: Wed Feb 10 15:27:28 2016 +0100 -- .../test/java/org/apache/flink/testutils/junit/RetryRule.java| 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/756cbaff/flink-core/src/test/java/org/apache/flink/testutils/junit/RetryRule.java -- diff --git a/flink-core/src/test/java/org/apache/flink/testutils/junit/RetryRule.java b/flink-core/src/test/java/org/apache/flink/testutils/junit/RetryRule.java index a4aff86..2b3a37a 100644 --- a/flink-core/src/test/java/org/apache/flink/testutils/junit/RetryRule.java +++ b/flink-core/src/test/java/org/apache/flink/testutils/junit/RetryRule.java @@ -113,7 +113,7 @@ public class RetryRule implements TestRule { break; // success } catch (Throwable t) { - LOG.debug(String.format("Test run failed (%d/%d).", + LOG.warn(String.format("Test run failed (%d/%d).", currentRun, timesOnFailure + 1), t); // Throw the failure if retried too often @@ -156,7 +156,7 @@ public class RetryRule implements TestRule { break; // success } catch (Throwable t) { - LOG.debug(String.format("Test run failed (%d/%d).", currentRun, timesOnFailure + 1), t); + LOG.warn(String.format("Test run failed (%d/%d).", currentRun, timesOnFailure + 1), t); if (!exceptionClass.isAssignableFrom(t.getClass()) || currentRun >= timesOnFailure) { // Throw the failure if retried too often, or if it is the wrong exception
[2/2] flink git commit: [FLINK-3384] [kafka] Add ClosableQueue for message exchanges between Kafka Threads
[FLINK-3384] [kafka] Add ClosableQueue for message exchanges between Kafka Threads Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/fd324ea7 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/fd324ea7 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/fd324ea7 Branch: refs/heads/master Commit: fd324ea72979cc3d4202ffa3ea174ec4cc9d153b Parents: 50bd65a Author: Stephan EwenAuthored: Wed Feb 10 14:51:10 2016 +0100 Committer: Stephan Ewen Committed: Wed Feb 10 22:15:32 2016 +0100 -- .../kafka/internals/ClosableBlockingQueue.java | 502 +++ .../internals/ClosableBlockingQueueTest.java| 603 +++ 2 files changed, 1105 insertions(+) -- http://git-wip-us.apache.org/repos/asf/flink/blob/fd324ea7/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java -- diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java new file mode 100644 index 000..856c2ad --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java @@ -0,0 +1,502 @@ +/* + * 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.streaming.connectors.kafka.internals; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +import static java.util.Objects.requireNonNull; + +/** + * A special form of blocking queue with two additions: + * + * The queue can be closed atomically when empty. Adding elements after the queue + * is closed fails. This allows queue consumers to atomically discover that no elements + * are available and mark themselves as shut down. + * The queue allows to poll batches of elements in one polling call. + * + * + * The queue has no capacity restriction and is safe for multiple producers and consumers. + * + * Note: Null elements are prohibited. + * + * @param The type of elements in the queue. + */ +public class ClosableBlockingQueue { + + /** The lock used to make queue accesses and open checks atomic */ + private final ReentrantLock lock; + + /** The condition on which blocking get-calls wait if the queue is empty */ + private final Condition nonEmpty; + + /** The deque of elements */ + private final ArrayDeque elements; + + /** Flag marking the status of the queue */ + private volatile boolean open; + + // + + /** +* Creates a new empty queue. +*/ + public ClosableBlockingQueue() { + this(10); + } + + /** +* Creates a new empty queue, reserving space for at least the specified number +* of elements. The queu can still grow, of more elements are added than the +* reserved space. +* +* @param initialSize The number of elements to reserve space for. +*/ + public ClosableBlockingQueue(int initialSize) { + this.lock = new ReentrantLock(true); + this.nonEmpty = this.lock.newCondition(); + + this.elements = new ArrayDeque<>(initialSize); + this.open = true; + + + } + + /** +* Creates a new queue
[1/2] flink git commit: [FLINK-3371] [api-breaking] Move TriggerResult and TriggerContext to dedicated classes
Repository: flink Updated Branches: refs/heads/master 8df0bbacb -> fd324ea72 [FLINK-3371] [api-breaking] Move TriggerResult and TriggerContext to dedicated classes This closes #1603 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/50bd65a5 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/50bd65a5 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/50bd65a5 Branch: refs/heads/master Commit: 50bd65a574776817a03dd32fd438cb2327447109 Parents: 8df0bba Author: Stephan EwenAuthored: Sun Feb 7 21:46:16 2016 +0100 Committer: Stephan Ewen Committed: Wed Feb 10 22:15:31 2016 +0100 -- .../examples/windowing/SessionWindowing.java| 3 +- .../api/windowing/assigners/GlobalWindows.java | 10 +- .../triggers/ContinuousEventTimeTrigger.java| 7 +- .../ContinuousProcessingTimeTrigger.java| 2 +- .../api/windowing/triggers/CountTrigger.java| 2 +- .../api/windowing/triggers/DeltaTrigger.java| 7 +- .../windowing/triggers/EventTimeTrigger.java| 5 +- .../triggers/ProcessingTimeTrigger.java | 5 +- .../api/windowing/triggers/PurgingTrigger.java | 4 +- .../api/windowing/triggers/Trigger.java | 102 +-- .../api/windowing/triggers/TriggerResult.java | 96 + .../windowing/EvictingWindowOperator.java | 5 +- .../windowing/NonKeyedWindowOperator.java | 34 --- .../operators/windowing/WindowOperator.java | 18 ++-- 14 files changed, 179 insertions(+), 121 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/50bd65a5/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java -- diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java index bd82800..e2df160 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java @@ -27,6 +27,7 @@ import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; import org.apache.flink.streaming.api.windowing.triggers.Trigger; +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult; import org.apache.flink.streaming.api.windowing.windows.GlobalWindow; import java.util.ArrayList; @@ -95,7 +96,7 @@ public class SessionWindowing { env.execute(); } - private static class SessionTrigger implements Trigger , GlobalWindow> { + private static class SessionTrigger extends Trigger , GlobalWindow> { private static final long serialVersionUID = 1L; http://git-wip-us.apache.org/repos/asf/flink/blob/50bd65a5/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java index d3eb2ac..a4d92cf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.windowing.triggers.Trigger; +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult; import org.apache.flink.streaming.api.windowing.windows.GlobalWindow; import java.util.Collection; @@ -67,15 +68,12 @@ public class GlobalWindows extends WindowAssigner
flink git commit: [FLINK-3355] [rocksdb backend] Allow passing options to the RocksDB backend.
Repository: flink Updated Branches: refs/heads/master 28c6254ee -> 9ee167949 [FLINK-3355] [rocksdb backend] Allow passing options to the RocksDB backend. This also cleans up the generics in the RocksDB state classes. This closes #1608 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9ee16794 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9ee16794 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9ee16794 Branch: refs/heads/master Commit: 9ee16794909d18aa84e8d0b738a6a447d11e6eeb Parents: 28c6254 Author: Stephan EwenAuthored: Mon Feb 8 19:55:29 2016 +0100 Committer: Stephan Ewen Committed: Tue Feb 9 11:03:09 2016 +0100 -- .../streaming/state/AbstractRocksDBState.java | 113 +-- .../contrib/streaming/state/OptionsFactory.java | 31 + .../streaming/state/RocksDBListState.java | 68 ++- .../streaming/state/RocksDBReducingState.java | 86 +++--- .../streaming/state/RocksDBStateBackend.java| 76 +++-- .../streaming/state/RocksDBValueState.java | 74 ++-- 6 files changed, 273 insertions(+), 175 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/9ee16794/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java -- diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java index 783332c..05e15e8 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java @@ -1,36 +1,38 @@ -/** - * 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 - * +/* + * 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.contrib.streaming.state; import org.apache.commons.io.FileUtils; + import org.apache.flink.api.common.state.State; import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.KvState; import org.apache.flink.runtime.state.KvStateSnapshot; import org.apache.flink.util.HDFSCopyFromLocal; import org.apache.flink.util.HDFSCopyToLocal; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; + import org.rocksdb.BackupEngine; import org.rocksdb.BackupableDBOptions; import org.rocksdb.Env; @@ -38,7 +40,7 @@ import org.rocksdb.Options; import org.rocksdb.RestoreOptions; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; -import org.rocksdb.StringAppendOperator; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,10 +62,9 @@ import static java.util.Objects.requireNonNull; * @param The type of the namespace. * @param The type of {@link State}. * @param The type of {@link StateDescriptor}. - * @param The type of the backend that snapshots this key/value state. */ -public abstract class AbstractRocksDBState ,
flink git commit: [FLINK-3373] [build] Bump version of transitine HTTP Components dependency to 4.4.4 (core) / 4.5.1 (client)
Repository: flink Updated Branches: refs/heads/master a4f0692e9 -> c57a7e910 [FLINK-3373] [build] Bump version of transitine HTTP Components dependency to 4.4.4 (core) / 4.5.1 (client) Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c57a7e91 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c57a7e91 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c57a7e91 Branch: refs/heads/master Commit: c57a7e910e1dcb45c2c3f7b7743a1c8b37ce7639 Parents: a4f0692 Author: Stephan EwenAuthored: Tue Feb 9 14:58:09 2016 +0100 Committer: Stephan Ewen Committed: Tue Feb 9 15:00:51 2016 +0100 -- pom.xml | 11 ++- 1 file changed, 2 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/c57a7e91/pom.xml -- diff --git a/pom.xml b/pom.xml index 1f1a772..81528c8 100644 --- a/pom.xml +++ b/pom.xml @@ -240,13 +240,6 @@ under the License. 1.7 - - - stax - stax-api - 1.0.1 - - com.esotericsoftware.kryo @@ -351,13 +344,13 @@ under the License. org.apache.httpcomponents httpcore - 4.2.5 + 4.4.4 org.apache.httpcomponents httpclient - 4.2.6 + 4.5.1
flink git commit: [FLINK-3268] [tests] Enhance stability of ZooKeeperTestEnvironment.deleteAll()
Repository: flink Updated Branches: refs/heads/master 7378f014b -> f8f747f22 [FLINK-3268] [tests] Enhance stability of ZooKeeperTestEnvironment.deleteAll() Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/f8f747f2 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/f8f747f2 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/f8f747f2 Branch: refs/heads/master Commit: f8f747f2290ad623bd8a4f0d8ff4708fada6792a Parents: 7378f01 Author: Stephan EwenAuthored: Wed Feb 3 22:13:25 2016 +0100 Committer: Stephan Ewen Committed: Thu Feb 4 15:49:15 2016 +0100 -- .../runtime/zookeeper/ZooKeeperTestEnvironment.java | 14 +- 1 file changed, 13 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/f8f747f2/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java index 94e1988..5aca687 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java @@ -137,7 +137,19 @@ public class ZooKeeperTestEnvironment { */ public void deleteAll() throws Exception { final String path = "/" + client.getNamespace(); - ZKPaths.deleteChildren(client.getZookeeperClient().getZooKeeper(), path, false); + + for (int attempt = 0; attempt < 3; attempt++) { + try { + ZKPaths.deleteChildren(client.getZookeeperClient().getZooKeeper(), path, false); + break; + } + catch (org.apache.zookeeper.KeeperException.NoNodeException e) { + // that seems all right. if one of the children we want to delete is + // actually already deleted, that's fine. + + // still, fall through the loop to delete the other children + } + } } }
[1/2] flink git commit: [streaming] [scala] Scala wrapper for DataStreamUtils
Repository: flink Updated Branches: refs/heads/master 69f7f6d92 -> 4852302a1 [streaming] [scala] Scala wrapper for DataStreamUtils This closes #1574 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4852302a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4852302a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4852302a Branch: refs/heads/master Commit: 4852302a1b0e2254ca0be283f363e70e867e7b54 Parents: 116337f Author: Márton BalassiAuthored: Tue Feb 2 14:18:05 2016 +0100 Committer: Stephan Ewen Committed: Thu Feb 4 17:29:08 2016 +0100 -- flink-contrib/flink-streaming-contrib/pom.xml | 137 +++ .../contrib/streaming/scala/utils/package.scala | 48 +++ 2 files changed, 185 insertions(+) -- http://git-wip-us.apache.org/repos/asf/flink/blob/4852302a/flink-contrib/flink-streaming-contrib/pom.xml -- diff --git a/flink-contrib/flink-streaming-contrib/pom.xml b/flink-contrib/flink-streaming-contrib/pom.xml index e608334..88ecf20 100644 --- a/flink-contrib/flink-streaming-contrib/pom.xml +++ b/flink-contrib/flink-streaming-contrib/pom.xml @@ -44,6 +44,11 @@ under the License. org.apache.flink + flink-streaming-scala_2.10 + ${project.version} + + + org.apache.flink flink-clients_2.10 ${project.version} @@ -79,4 +84,136 @@ under the License. + + + + + net.alchim31.maven + scala-maven-plugin + 3.1.4 + + + + scala-compile-first + process-resources + + compile + + + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + -Xms128m + -Xmx512m + + + + org.scalamacros + paradise_${scala.version} + ${scala.macros.version} + + + + + + + + org.apache.maven.plugins + maven-eclipse-plugin + 2.8 + + true + + org.scala-ide.sdt.core.scalanature + org.eclipse.jdt.core.javanature + + + org.scala-ide.sdt.core.scalabuilder + + + org.scala-ide.sdt.launching.SCALA_CONTAINER + org.eclipse.jdt.launching.JRE_CONTAINER + + + org.scala-lang:scala-library + org.scala-lang:scala-compiler + + + **/*.scala +
[4/4] flink git commit: [hotfix] Minor code cleanups in AbstractStateBackend
[hotfix] Minor code cleanups in AbstractStateBackend Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d93b154d Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d93b154d Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d93b154d Branch: refs/heads/master Commit: d93b154d41045e89dcc2238885db83fd947cd104 Parents: b6bfcf0 Author: Stephan EwenAuthored: Thu Feb 11 22:13:37 2016 +0100 Committer: Stephan Ewen Committed: Fri Feb 12 18:51:01 2016 +0100 -- .../apache/flink/runtime/state/AbstractStateBackend.java | 10 +- 1 file changed, 5 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/d93b154d/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java index e989af3..d0c4f82 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java @@ -123,7 +123,7 @@ public abstract class AbstractStateBackend implements java.io.Serializable { * @param The type of the namespace. * @param The type of the value that the {@code ValueState} can store. */ - abstract protected ValueState createValueState(TypeSerializer namespaceSerializer, ValueStateDescriptor stateDesc) throws Exception; + protected abstract ValueState createValueState(TypeSerializer namespaceSerializer, ValueStateDescriptor stateDesc) throws Exception; /** * Creates and returns a new {@link ListState}. @@ -134,7 +134,7 @@ public abstract class AbstractStateBackend implements java.io.Serializable { * @param The type of the namespace. * @param The type of the values that the {@code ListState} can store. */ - abstract protected ListState createListState(TypeSerializer namespaceSerializer, ListStateDescriptor stateDesc) throws Exception; + protected abstract ListState createListState(TypeSerializer namespaceSerializer, ListStateDescriptor stateDesc) throws Exception; /** * Creates and returns a new {@link ReducingState}. @@ -145,7 +145,7 @@ public abstract class AbstractStateBackend implements java.io.Serializable { * @param The type of the namespace. * @param The type of the values that the {@code ListState} can store. */ - abstract protected ReducingState createReducingState(TypeSerializer namespaceSerializer, ReducingStateDescriptor stateDesc) throws Exception; + protected abstract ReducingState createReducingState(TypeSerializer namespaceSerializer, ReducingStateDescriptor stateDesc) throws Exception; /** * Sets the current key that is used for partitioned state. @@ -170,7 +170,7 @@ public abstract class AbstractStateBackend implements java.io.Serializable { * * @param stateDescriptor The state identifier for the state. This contains name * and can create a default state value. -* @param The type of the key. + * @param The type of the namespace. * @param The type of the state. * @@ -179,7 +179,7 @@ public abstract class AbstractStateBackend implements java.io.Serializable { * @throws Exception Exceptions may occur during initialization of the state and should be forwarded. */ @SuppressWarnings({"rawtypes", "unchecked"}) - public S getPartitionedState(final N namespace, final TypeSerializer namespaceSerializer, final StateDescriptor stateDescriptor) throws Exception { + publicS getPartitionedState(final N namespace, final TypeSerializer namespaceSerializer, final StateDescriptor stateDescriptor) throws Exception { if (keySerializer == null) { throw new Exception("State key serializer has not been configured in the config. " +
[2/4] flink git commit: [FLINK-2991] Add Folding State and use in WindowOperator
http://git-wip-us.apache.org/repos/asf/flink/blob/94cba899/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java index 88e619a..0d01733 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.functions.FoldFunction; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.state.FoldingStateDescriptor; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -36,10 +37,9 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction; import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator; import org.apache.flink.streaming.api.functions.aggregation.SumAggregator; -import org.apache.flink.streaming.api.functions.windowing.FoldWindowFunction; +import org.apache.flink.streaming.api.functions.windowing.FoldApplyWindowFunction; +import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction; import org.apache.flink.streaming.api.functions.windowing.ReduceApplyWindowFunction; -import org.apache.flink.streaming.api.functions.windowing.ReduceIterableWindowFunction; -import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction; import org.apache.flink.streaming.api.functions.windowing.WindowFunction; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows; @@ -71,7 +71,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; * If an {@link Evictor} is specified it will be used to evict elements from the window after * evaluation was triggered by the {@code Trigger} but before the actual evaluation of the window. * When using an evictor window performance will degrade significantly, since - * pre-aggregation of window results cannot be used. + * incremental aggregation of window results cannot be used. * * * Note that the {@code WindowedStream} is purely and API construct, during runtime @@ -120,7 +120,7 @@ public class WindowedStream{ * * * Note: When using an evictor window performance will degrade significantly, since -* pre-aggregation of window results cannot be used. +* incremental aggregation of window results cannot be used. */ @PublicEvolving public WindowedStream evictor(Evictor evictor) { @@ -137,13 +137,14 @@ public class WindowedStream { * Applies a reduce function to the window. The window function is called for each evaluation * of the window for each key individually. The output of the reduce function is interpreted * as a regular non-windowed stream. +* * -* This window will try and pre-aggregate data as much as the window policies permit. For example, -* tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per -* key is stored. Sliding time windows will pre-aggregate on the granularity of the slide interval, +* This window will try and incrementally aggregate data as much as the window policies permit. +* For example, tumbling time windows can aggregate the data, meaning that only one element per +* key is stored. Sliding time windows will aggregate on the granularity of the slide interval, * so a few elements are stored per key (one per slide interval). -* Custom windows may not be able to pre-aggregate, or may need to store extra values in an -* aggregation tree. +* Custom windows may not be able to incrementally aggregate, or may need to store extra values +* in an aggregation tree. * * @param function The reduce function. * @return The data stream that is the result of applying the reduce function to the window. @@ -159,48 +160,14 @@ public class WindowedStream { function = input.getExecutionEnvironment().clean(function); String callLocation = Utils.getCallLocationName(); -
[1/3] flink git commit: [FLINK-3358] [FLINK-3351] [rocksdb] Add simple constructors and automatic temp path configuration
Repository: flink Updated Branches: refs/heads/master dcea46e89 -> edae79340 [FLINK-3358] [FLINK-3351] [rocksdb] Add simple constructors and automatic temp path configuration This adds constructors that only take a backup dir URI and use it to initialize both the RocksDB file backups and the FileSystem state backend for non-partitioned state. Also, the RocksDBStateBackend now automatically picks up the TaskManager's temp directories, if no local storage directories are explicitly configured. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/edae7934 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/edae7934 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/edae7934 Branch: refs/heads/master Commit: edae79340dd486915d25109cbdc1485accae665a Parents: be72758 Author: Stephan EwenAuthored: Thu Feb 11 21:30:36 2016 +0100 Committer: Stephan Ewen Committed: Thu Feb 11 21:34:03 2016 +0100 -- .../streaming/state/RocksDBStateBackend.java| 241 ++-- .../state/RocksDBStateBackendConfigTest.java| 280 +++ .../state/RocksDBStateBackendTest.java | 4 +- .../state/filesystem/FsStateBackend.java| 99 --- .../EventTimeWindowCheckpointingITCase.java | 13 +- 5 files changed, 566 insertions(+), 71 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/edae7934/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java -- diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java index eddd8c0..5b16e86 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java @@ -18,7 +18,12 @@ package org.apache.flink.contrib.streaming.state; import java.io.File; +import java.io.IOException; import java.io.Serializable; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.state.ListState; @@ -28,13 +33,17 @@ import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.api.common.state.StateBackend; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.rocksdb.Options; import org.rocksdb.StringAppendOperator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static java.util.Objects.requireNonNull; @@ -54,20 +63,35 @@ import static java.util.Objects.requireNonNull; public class RocksDBStateBackend extends AbstractStateBackend { private static final long serialVersionUID = 1L; - /** Base path for RocksDB directory. */ - private final String dbBasePath; - - /** The checkpoint directory that we snapshot RocksDB backups to. */ - private final String checkpointDirectory; + private static final Logger LOG = LoggerFactory.getLogger(RocksDBStateBackend.class); + + + /** The checkpoint directory that copy the RocksDB backups to. */ + private final Path checkpointDirectory; + /** The state backend that stores the non-partitioned state */ + private final AbstractStateBackend nonPartitionedStateBackend; + + /** Operator identifier that is used to uniqueify the RocksDB storage path. */ private String operatorIdentifier; /** JobID for uniquifying backup paths. */ private JobID jobId; + - private AbstractStateBackend backingStateBackend; + // DB storage directories + + /** Base paths for RocksDB directory, as configured. May be null. */ + private Path[] dbBasePaths; + /** Base paths for RocksDB directory, as initialized */ + private File[] dbStorageDirectories; + + private int nextDirectory; + + // RocksDB options + /** The pre-configured option settings */ private
[3/3] flink git commit: [FLINK-3271] [build] Don't exclude jetty-util dependency from the Hadoop dependencies
[FLINK-3271] [build] Don't exclude jetty-util dependency from the Hadoop dependencies This closes #1543 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/82c7383a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/82c7383a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/82c7383a Branch: refs/heads/master Commit: 82c7383a67a8349d05dc98780667b3a47ab3cc54 Parents: dcea46e Author: Abhishek AgarwalAuthored: Sun Jan 24 20:22:45 2016 +0530 Committer: Stephan Ewen Committed: Thu Feb 11 21:34:03 2016 +0100 -- .../flink-shaded-hadoop1/pom.xml| 12 -- .../flink-shaded-hadoop2/pom.xml| 41 +--- 2 files changed, 1 insertion(+), 52 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/82c7383a/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml -- diff --git a/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml index 15082aa..e8634f5 100644 --- a/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml +++ b/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml @@ -69,10 +69,6 @@ under the License. jsp-2.1 - org.mortbay.jetty - jetty-util - - org.eclipse.jdt core @@ -81,14 +77,6 @@ under the License. servlet-api - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - com.sun.jersey jersey-core http://git-wip-us.apache.org/repos/asf/flink/blob/82c7383a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml -- diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml index 5eb8043..c642653 100644 --- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml +++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml @@ -68,10 +68,7 @@ under the License. org.mortbay.jetty jsp-2.1 - - org.mortbay.jetty - jetty-util - + org.eclipse.jdt core @@ -81,10 +78,6 @@ under the License. jetty - org.mortbay.jetty - jetty-util - - com.sun.jersey jersey-json @@ -185,10 +178,6 @@ under the License. jsp-2.1 - org.mortbay.jetty - jetty-util - - org.eclipse.jdt core @@ -197,10 +186,6 @@ under the License. jetty - org.mortbay.jetty - jetty-util - - com.sun.jersey jersey-json @@ -301,10 +286,6 @@ under the License. jsp-2.1 -
[2/3] flink git commit: [FLINK-3389] [rocksdb] Add pre-defined option profiles.
[FLINK-3389] [rocksdb] Add pre-defined option profiles. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/be72758d Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/be72758d Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/be72758d Branch: refs/heads/master Commit: be72758d1104400c8a48554d717c5b8cea5b3617 Parents: 82c7383 Author: Stephan EwenAuthored: Thu Feb 11 15:30:56 2016 +0100 Committer: Stephan Ewen Committed: Thu Feb 11 21:34:03 2016 +0100 -- .../contrib/streaming/state/OptionsFactory.java | 32 ++- .../streaming/state/PredefinedOptions.java | 91 .../streaming/state/RocksDBStateBackend.java| 76 ++-- 3 files changed, 190 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/be72758d/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactory.java -- diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactory.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactory.java index 73b1e5d..3e52f1f 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactory.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactory.java @@ -24,8 +24,36 @@ import org.rocksdb.Options; * A factory for {@link Options} to be passed to the {@link RocksDBStateBackend}. * Options have to be created lazily by this factory, because the {@code Options} * class is not serializable and holds pointers to native code. + * + * A typical pattern to use this OptionsFactory is as follows: + * + * Java 8: + * {@code + * rocksDbBackend.setOptions( (currentOptions) -> currentOptions.setMaxOpenFiles(1024) ); + * } + * + * Java 7: + * {@code + * rocksDbBackend.setOptions(new OptionsFactory() { + * + * public Options setOptions(Options currentOptions) { + * return currentOptions.setMaxOpenFiles(1024); + * } + * }) + * } */ public interface OptionsFactory extends java.io.Serializable { - - Options createOptions(); + + /** +* This method should set the additional options on top of the current options object. +* The current options object may contain pre-defined options based on flags that have +* been configured on the state backend. +* +* It is important to set the options on the current object and return the result from +* the setter methods, otherwise the pre-defined options may get lost. +* +* @param currentOptions The options object with the pre-defined options. +* @return The options object on which the additional options are set. +*/ + Options createOptions(Options currentOptions); } http://git-wip-us.apache.org/repos/asf/flink/blob/be72758d/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java -- diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java new file mode 100644 index 000..383f043 --- /dev/null +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java @@ -0,0 +1,91 @@ +/* + * 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.contrib.streaming.state; + +import org.rocksdb.CompactionStyle; +import org.rocksdb.Options; + +/** + * The {@code PredefinedOptions} are configuration settings for the {@link
flink git commit: [FLINK-3338] [kafka] Use proper classloader when cloning the deserialization schema.
Repository: flink Updated Branches: refs/heads/master fe0c3b539 -> 2eb2a0ef3 [FLINK-3338] [kafka] Use proper classloader when cloning the deserialization schema. This closes #1590 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2eb2a0ef Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2eb2a0ef Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2eb2a0ef Branch: refs/heads/master Commit: 2eb2a0ef352f75a65a45a5a247450ae61ae5ab17 Parents: fe0c3b5 Author: Stephan EwenAuthored: Thu Feb 4 21:14:39 2016 +0100 Committer: Stephan Ewen Committed: Fri Feb 5 14:07:42 2016 +0100 -- .../apache/flink/util/InstantiationUtil.java| 29 ++-- .../connectors/kafka/FlinkKafkaConsumer08.java | 3 +- .../kafka/internals/LegacyFetcher.java | 26 -- .../kafka/testutils/MockRuntimeContext.java | 2 +- 4 files changed, 48 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/2eb2a0ef/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java -- diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java index 1c6896f..e2439ca 100644 --- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java @@ -310,8 +310,33 @@ public final class InstantiationUtil { * @throws ClassNotFoundException */ public static T clone(T obj) throws IOException, ClassNotFoundException { - final byte[] serializedObject = serializeObject(obj); - return deserializeObject(serializedObject, obj.getClass().getClassLoader()); + if (obj == null) { + return null; + } else { + return clone(obj, obj.getClass().getClassLoader()); + } + } + + /** +* Clones the given serializable object using Java serialization, using the given classloader to +* resolve the cloned classes. +* +* @param obj Object to clone +* @param classLoader The classloader to resolve the classes during deserialization. +* @param Type of the object to clone +* +* @return Cloned object +* +* @throws IOException +* @throws ClassNotFoundException +*/ + public static T clone(T obj, ClassLoader classLoader) throws IOException, ClassNotFoundException { + if (obj == null) { + return null; + } else { + final byte[] serializedObject = serializeObject(obj); + return deserializeObject(serializedObject, classLoader); + } } // http://git-wip-us.apache.org/repos/asf/flink/blob/2eb2a0ef/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java -- diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java index 543e0ff..bdea37f 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java @@ -251,7 +251,8 @@ public class FlinkKafkaConsumer08 extends FlinkKafkaConsumerBase { } // create fetcher - fetcher = new LegacyFetcher(this.subscribedPartitions, props, getRuntimeContext().getTaskName()); + fetcher = new LegacyFetcher(this.subscribedPartitions, props, + getRuntimeContext().getTaskName(), getRuntimeContext().getUserCodeClassLoader()); // offset handling offsetHandler = new ZookeeperOffsetHandler(props); http://git-wip-us.apache.org/repos/asf/flink/blob/2eb2a0ef/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java -- diff --git
[6/9] flink git commit: [hotfix] typo IllegalArgumentException
[hotfix] typo IllegalArgumentException This closes #1602 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/5c47f385 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/5c47f385 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/5c47f385 Branch: refs/heads/master Commit: 5c47f3854938fc789ac0fd0867451f4def155c90 Parents: d51bec1 Author: Andrea SellaAuthored: Mon Feb 8 16:38:38 2016 +0100 Committer: Stephan Ewen Committed: Mon Feb 8 20:18:19 2016 +0100 -- .../java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/5c47f385/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java -- diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java index 84eb309..132edc4 100644 --- a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java +++ b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java @@ -357,10 +357,10 @@ public class JDBCInputFormat extends RichInputFormat
[5/9] flink git commit: [FLINK-3334] [conf] Include year-month-day in the log output
[FLINK-3334] [conf] Include year-month-day in the log output Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d51bec15 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d51bec15 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d51bec15 Branch: refs/heads/master Commit: d51bec1524a9832a7e6c0f6c5d5be5a42712d365 Parents: 457cb14 Author: Stephan EwenAuthored: Mon Feb 8 17:38:52 2016 +0100 Committer: Stephan Ewen Committed: Mon Feb 8 19:59:31 2016 +0100 -- flink-dist/src/main/flink-bin/conf/log4j-cli.properties| 4 ++-- .../src/main/flink-bin/conf/log4j-yarn-session.properties | 2 +- flink-dist/src/main/flink-bin/conf/log4j.properties| 2 +- flink-dist/src/main/flink-bin/conf/logback-yarn.xml| 6 +++--- flink-dist/src/main/flink-bin/conf/logback.xml | 5 +++-- 5 files changed, 10 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/d51bec15/flink-dist/src/main/flink-bin/conf/log4j-cli.properties -- diff --git a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties index 9c56e61..acb9d1a 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties @@ -23,7 +23,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.file=${log.file} log4j.appender.file.append=false log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n +log4j.appender.file.layout.ConversionPattern=%d{-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n # Log output from org.apache.flink.yarn to the console. This is used by the @@ -34,7 +34,7 @@ log4j.logger.org.apache.hadoop=INFO, console log4j.appender.console=org.apache.log4j.ConsoleAppender log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n +log4j.appender.console.layout.ConversionPattern=%d{-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n # suppress the warning that hadoop native libraries are not loaded (irrelevant for the client) log4j.logger.org.apache.hadoop.util.NativeCodeLoader=OFF http://git-wip-us.apache.org/repos/asf/flink/blob/d51bec15/flink-dist/src/main/flink-bin/conf/log4j-yarn-session.properties -- diff --git a/flink-dist/src/main/flink-bin/conf/log4j-yarn-session.properties b/flink-dist/src/main/flink-bin/conf/log4j-yarn-session.properties index 1f49676..07f65a7 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j-yarn-session.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j-yarn-session.properties @@ -21,7 +21,7 @@ log4j.rootLogger=INFO, stdout # Log all infos in the given file log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n +log4j.appender.stdout.layout.ConversionPattern=%d{-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n # suppress the irrelevant (wrong) warnings from the netty channel handler log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, stdout http://git-wip-us.apache.org/repos/asf/flink/blob/d51bec15/flink-dist/src/main/flink-bin/conf/log4j.properties -- diff --git a/flink-dist/src/main/flink-bin/conf/log4j.properties b/flink-dist/src/main/flink-bin/conf/log4j.properties index adcff38..97ec653 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j.properties @@ -23,7 +23,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.file=${log.file} log4j.appender.file.append=false log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n +log4j.appender.file.layout.ConversionPattern=%d{-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n # suppress the irrelevant (wrong) warnings from the netty channel handler log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, file http://git-wip-us.apache.org/repos/asf/flink/blob/d51bec15/flink-dist/src/main/flink-bin/conf/logback-yarn.xml -- diff --git a/flink-dist/src/main/flink-bin/conf/logback-yarn.xml b/flink-dist/src/main/flink-bin/conf/logback-yarn.xml index c5511b3..636b78a 100644 ---
[2/9] flink git commit: [FLINK-3363] [jobmanager] Properly shut down executor thread pool when JobManager shuts down
[FLINK-3363] [jobmanager] Properly shut down executor thread pool when JobManager shuts down Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/a277543c Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/a277543c Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/a277543c Branch: refs/heads/master Commit: a277543c57f7c633e0f8b610b241eac5a95aa7cc Parents: af3e689 Author: Stephan EwenAuthored: Mon Feb 8 13:18:50 2016 +0100 Committer: Stephan Ewen Committed: Mon Feb 8 16:57:57 2016 +0100 -- .../flink/runtime/jobmanager/JobManager.scala | 42 +--- .../runtime/minicluster/FlinkMiniCluster.scala | 5 +- .../JobManagerLeaderElectionTest.java | 16 ++- .../testingUtils/TestingJobManager.scala| 17 +--- .../LocalFlinkMiniClusterITCase.java| 101 --- .../flink/yarn/TestingYarnJobManager.scala | 9 +- .../org/apache/flink/yarn/YarnJobManager.scala | 14 ++- 7 files changed, 149 insertions(+), 55 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/a277543c/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala -- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index bc7c134..d96575f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -21,6 +21,7 @@ package org.apache.flink.runtime.jobmanager import java.io.{File, IOException} import java.net.{BindException, ServerSocket, UnknownHostException, InetAddress, InetSocketAddress} import java.util.UUID +import java.util.concurrent.ExecutorService import akka.actor.Status.Failure import akka.actor._ @@ -90,7 +91,7 @@ import scala.language.postfixOps * is indicated by [[CancellationSuccess]] and a failure by [[CancellationFailure]] * * - [[UpdateTaskExecutionState]] is sent by a TaskManager to update the state of an - ExecutionVertex contained in the [[ExecutionGraph]]. + * ExecutionVertex contained in the [[ExecutionGraph]]. * A successful update is acknowledged by true and otherwise false. * * - [[RequestNextInputSplit]] requests the next input split for a running task on a @@ -102,7 +103,7 @@ import scala.language.postfixOps */ class JobManager( protected val flinkConfiguration: Configuration, -protected val executionContext: ExecutionContext, +protected val executorService: ExecutorService, protected val instanceManager: InstanceManager, protected val scheduler: FlinkScheduler, protected val libraryCacheManager: BlobLibraryCacheManager, @@ -121,6 +122,15 @@ class JobManager( override val log = Logger(getClass) + /** The extra execution context, for futures, with a custom logging reporter */ + protected val executionContext: ExecutionContext = ExecutionContext.fromExecutor( +executorService, +(t: Throwable) => { + if (!context.system.isTerminated) { +log.error("Executor could not execute task", t) + } +}) + /** Either running or not yet archived jobs (session hasn't been ended). */ protected val currentJobs = scala.collection.mutable.HashMap[JobID, (ExecutionGraph, JobInfo)]() @@ -246,6 +256,9 @@ class JobManager( case e: IOException => log.error("Could not properly shutdown the library cache manager.", e) } +// shut down the extra thread pool for futures +executorService.shutdown() + log.debug(s"Job manager ${self.path} is completely stopped.") } @@ -1503,7 +1516,8 @@ class JobManager( /** * Updates the accumulators reported from a task manager via the Heartbeat message. - * @param accumulators list of accumulator snapshots +* +* @param accumulators list of accumulator snapshots */ private def updateAccumulators(accumulators : Seq[AccumulatorSnapshot]) = { accumulators foreach { @@ -2016,7 +2030,7 @@ object JobManager { def createJobManagerComponents( configuration: Configuration, leaderElectionServiceOption: Option[LeaderElectionService]) : -(ExecutionContext, +(ExecutorService, InstanceManager, FlinkScheduler, BlobLibraryCacheManager, @@ -2064,17 +2078,19 @@ object JobManager { } } -val executionContext = ExecutionContext.fromExecutor(new ForkJoinPool()) + var blobServer: BlobServer = null var instanceManager: InstanceManager = null var scheduler: FlinkScheduler = null var libraryCacheManager:
[7/9] flink git commit: [FLINK-3357] [core] Drop AbstractID#toShortString()
[FLINK-3357] [core] Drop AbstractID#toShortString() This closes #1601 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/28feede7 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/28feede7 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/28feede7 Branch: refs/heads/master Commit: 28feede7d40dc73ec861cf93393650b8b10afc3a Parents: 5c47f38 Author: Ufuk CelebiAuthored: Mon Feb 8 16:05:27 2016 +0100 Committer: Stephan Ewen Committed: Mon Feb 8 20:18:20 2016 +0100 -- .../streaming/state/RocksDBStateBackend.java| 4 +-- .../contrib/streaming/state/DbStateBackend.java | 35 ++-- .../streaming/state/DbStateBackendTest.java | 10 +++--- .../java/org/apache/flink/util/AbstractID.java | 15 - ...taskExecutionAttemptAccumulatorsHandler.java | 2 +- .../InputGateDeploymentDescriptor.java | 2 +- .../io/network/partition/ResultPartitionID.java | 2 +- 7 files changed, 28 insertions(+), 42 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/28feede7/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java -- diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java index aaaeea4..eefa4a9 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java @@ -81,11 +81,11 @@ public class RocksDBStateBackend extends AbstractStateBackend { } private File getDbPath(String stateName) { - return new File(new File(new File(new File(dbBasePath), jobId.toShortString()), operatorIdentifier), stateName); + return new File(new File(new File(new File(dbBasePath), jobId.toString()), operatorIdentifier), stateName); } private String getCheckpointPath(String stateName) { - return checkpointDirectory + "/" + jobId.toShortString() + "/" + operatorIdentifier + "/" + stateName; + return checkpointDirectory + "/" + jobId.toString() + "/" + operatorIdentifier + "/" + stateName; } @Override http://git-wip-us.apache.org/repos/asf/flink/blob/28feede7/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/state/DbStateBackend.java -- diff --git a/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/state/DbStateBackend.java b/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/state/DbStateBackend.java index 1d1ccd7..5162983 100644 --- a/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/state/DbStateBackend.java +++ b/flink-contrib/flink-streaming-contrib/src/main/java/org/apache/flink/contrib/streaming/state/DbStateBackend.java @@ -17,14 +17,6 @@ package org.apache.flink.contrib.streaming.state; -import java.io.Serializable; -import java.sql.Connection; -import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.Random; -import java.util.concurrent.Callable; - import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ReducingState; @@ -42,6 +34,14 @@ import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Random; +import java.util.concurrent.Callable; + import static org.apache.flink.contrib.streaming.state.SQLRetrier.retry; /** @@ -76,6 +76,8 @@ public class DbStateBackend extends AbstractStateBackend { private transient Environment env; + private transient String appId; + // -- private final DbBackendConfig dbConfig; @@ -159,19 +161,14 @@ public class DbStateBackend extends AbstractStateBackend { // store the checkpoint id and timestamp for bookkeeping long handleId = rnd.nextLong(); - // We use the
[4/9] flink git commit: [FLINK-3337] [runtime] mvn test fails on flink-runtime because curator classes not found
[FLINK-3337] [runtime] mvn test fails on flink-runtime because curator classes not found Removes curator dependency exclusions from flink-runtime. This resolves NoClassDefFoundError exceptions when running `mvn test`. This partialy reverts e31a4d8. This closes #1596 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4cc4f602 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4cc4f602 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4cc4f602 Branch: refs/heads/master Commit: 4cc4f6022c5935dffc52a37de7ebd5b272e95487 Parents: 22423e2 Author: Greg HoganAuthored: Fri Feb 5 16:41:24 2016 -0500 Committer: Stephan Ewen Committed: Mon Feb 8 16:57:57 2016 +0100 -- flink-runtime/pom.xml | 10 -- 1 file changed, 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/4cc4f602/flink-runtime/pom.xml -- diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index d8693e7..c6b6d1d 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -323,11 +323,6 @@ under the License. **/TestData.java **/TestInstanceListener.java - - org.apache.curator:curator-recipes - org.apache.curator:curator-client - org.apache.curator:curator-framework - WARN @@ -341,11 +336,6 @@ under the License. **/TestData.java - - org.apache.curator:curator-recipes - org.apache.curator:curator-client - org.apache.curator:curator-framework - WARN
[8/9] flink git commit: [hotfix] Update comments in 'ChainingStrategy' and remove outdated 'FORCE_ALWAYS' constant
[hotfix] Update comments in 'ChainingStrategy' and remove outdated 'FORCE_ALWAYS' constant Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/e9c83ea2 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/e9c83ea2 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/e9c83ea2 Branch: refs/heads/master Commit: e9c83ea2c36decf02d0ea9c2b76b0fd50606b51b Parents: 28feede Author: Stephan EwenAuthored: Mon Feb 8 15:08:09 2016 +0100 Committer: Stephan Ewen Committed: Mon Feb 8 20:19:25 2016 +0100 -- .../api/graph/StreamingJobGraphGenerator.java | 9 ++ .../api/operators/ChainingStrategy.java | 30 +++- 2 files changed, 19 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/e9c83ea2/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index fd75ba7..c0d2856 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -398,15 +398,12 @@ public class StreamingJobGraphGenerator { && headOperator != null && upStreamVertex.getSlotSharingID() == downStreamVertex.getSlotSharingID() && upStreamVertex.getSlotSharingID() != -1 - && (outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS || - outOperator.getChainingStrategy() == ChainingStrategy.FORCE_ALWAYS) + && outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS && (headOperator.getChainingStrategy() == ChainingStrategy.HEAD || - headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS || - headOperator.getChainingStrategy() == ChainingStrategy.FORCE_ALWAYS) + headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS) && (edge.getPartitioner() instanceof ForwardPartitioner) && upStreamVertex.getParallelism() == downStreamVertex.getParallelism() - && (streamGraph.isChainingEnabled() || - outOperator.getChainingStrategy() == ChainingStrategy.FORCE_ALWAYS); + && streamGraph.isChainingEnabled(); } private void setSlotSharing() { http://git-wip-us.apache.org/repos/asf/flink/blob/e9c83ea2/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java index 18e8858..1bf3259 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java @@ -18,23 +18,22 @@ package org.apache.flink.streaming.api.operators; - /** - * Defines the chaining scheme for the operator. - * By default {@link #ALWAYS} is used, which means operators will be eagerly chained whenever possible. + * Defines the chaining scheme for the operator. When an operator is chained to the + * predecessor, it means that they run in the same thread. They become one operator + * consisting of multiple steps. + * + * The default value used by the {@link StreamOperator} is {@link #HEAD}, which means that + * the operator is not chained to its predecessor. Most operators override this with + * {@link #ALWAYS}, meaning they will be chained to predecessors whenever possible. */ public enum ChainingStrategy { - /** -* Chaining will happen even if chaining is disabled on the execution environment. -* This should only be used by system-level operators, not operators implemented by users. -*/ - FORCE_ALWAYS, - /** -* Operators will be eagerly chained whenever possible, for -* maximal performance. It is generally a good
[1/9] flink git commit: [FLINK-3365] [taskmanager] Properly shut down cleanup timer thread
Repository: flink Updated Branches: refs/heads/master 22423e23f -> 28c6254ee [FLINK-3365] [taskmanager] Properly shut down cleanup timer thread Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/457cb145 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/457cb145 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/457cb145 Branch: refs/heads/master Commit: 457cb14537c0e6f89cdbcc0564054f86bfbbb3eb Parents: a277543 Author: Stephan EwenAuthored: Mon Feb 8 13:21:42 2016 +0100 Committer: Stephan Ewen Committed: Mon Feb 8 16:57:57 2016 +0100 -- .../execution/librarycache/BlobLibraryCacheManager.java | 7 +-- 1 file changed, 5 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/457cb145/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java index 98caac4..fcffd39 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java @@ -68,14 +68,16 @@ public final class BlobLibraryCacheManager extends TimerTask implements LibraryC /** The blob service to download libraries */ private final BlobService blobService; + private final Timer cleanupTimer; + // public BlobLibraryCacheManager(BlobService blobService, long cleanupInterval) { this.blobService = blobService; // Initializing the clean up task - Timer timer = new Timer(true); - timer.schedule(this, cleanupInterval); + this.cleanupTimer = new Timer(true); + this.cleanupTimer.schedule(this, cleanupInterval); } // @@ -199,6 +201,7 @@ public final class BlobLibraryCacheManager extends TimerTask implements LibraryC @Override public void shutdown() throws IOException{ blobService.shutdown(); + cleanupTimer.cancel(); } /**
[5/5] flink git commit: [FLINK-3265] [rabbitmq] Fix concurrency bug in RabbitMQ
[FLINK-3265] [rabbitmq] Fix concurrency bug in RabbitMQ This closes #1534 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6b01a890 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6b01a890 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6b01a890 Branch: refs/heads/master Commit: 6b01a89020f2de3f7710cf72336291b1e8ca8562 Parents: d97fcda Author: Robert MetzgerAuthored: Thu Jan 21 12:22:21 2016 +0100 Committer: Stephan Ewen Committed: Thu Jan 28 14:43:03 2016 +0100 -- .../connectors/rabbitmq/RMQSource.java | 4 +- .../connectors/rabbitmq/RMQSourceTest.java | 79 .../source/MessageAcknowledgingSourceBase.java | 51 +++-- ...ltipleIdsMessageAcknowledgingSourceBase.java | 24 +++--- 4 files changed, 124 insertions(+), 34 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6b01a890/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java -- diff --git a/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java index 09bb07c..59bc057 100644 --- a/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java +++ b/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java @@ -196,7 +196,9 @@ public class RMQSource extends MultipleIdsMessageAcknowledgingSourceBasehttp://git-wip-us.apache.org/repos/asf/flink/blob/6b01a890/flink-streaming-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java -- diff --git a/flink-streaming-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java b/flink-streaming-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java index aa19e5d..0a3de84 100644 --- a/flink-streaming-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java +++ b/flink-streaming-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java @@ -23,6 +23,7 @@ import com.rabbitmq.client.Envelope; import com.rabbitmq.client.QueueingConsumer; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.configuration.Configuration; @@ -31,6 +32,7 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -103,6 +105,83 @@ public class RMQSourceTest { sourceThread.join(); } + /** +* Make sure concurrent access to snapshotState() and notifyCheckpointComplete() don't cause +* an issue. +* +* Without proper synchronization, the test will fail with a concurrent modification exception +* +*/ + @Test + public void testConcurrentAccess() throws Exception { + source.autoAck = false; + sourceThread.start(); + + final Tuple1 error = new Tuple1<>(null); + + Thread.sleep(5); + + Thread snapshotThread = new Thread(new Runnable() { + public long id = 0; + + @Override + public void run() { + while (!Thread.interrupted()) { + try { + source.snapshotState(id++, 0); + } catch (Exception e) { + error.f0 = e; + break; // stop thread + } + } + } + }); + + Thread notifyThread =
[3/5] flink git commit: [FLINK-3242] Also Set User-specified StateBackend without Checkpointing
[FLINK-3242] Also Set User-specified StateBackend without Checkpointing Before, the user-specified StateBackedn would not be set when generating the JobGraph if checkpointing was disabled. This closes #1516 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/83b88c2c Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/83b88c2c Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/83b88c2c Branch: refs/heads/master Commit: 83b88c2c606f0d36bc04a7250629eb00516af919 Parents: f6d2ce9 Author: Aljoscha KrettekAuthored: Mon Jan 18 11:53:31 2016 +0100 Committer: Stephan Ewen Committed: Thu Jan 28 14:30:28 2016 +0100 -- .../api/graph/StreamingJobGraphGenerator.java | 2 +- .../runtime/state/StateBackendITCase.java | 134 +++ 2 files changed, 135 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/83b88c2c/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 50c6a15..56b16a4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -317,10 +317,10 @@ public class StreamingJobGraphGenerator { final CheckpointConfig ceckpointCfg = streamGraph.getCheckpointConfig(); + config.setStateBackend(streamGraph.getStateBackend()); config.setCheckpointingEnabled(ceckpointCfg.isCheckpointingEnabled()); if (ceckpointCfg.isCheckpointingEnabled()) { config.setCheckpointMode(ceckpointCfg.getCheckpointingMode()); - config.setStateBackend(streamGraph.getStateBackend()); } else { // the "at-least-once" input handler is slightly cheaper (in the absence of checkpoints), http://git-wip-us.apache.org/repos/asf/flink/blob/83b88c2c/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java new file mode 100644 index 000..cdfef85 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java @@ -0,0 +1,134 @@ +/* + * 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.streaming.runtime.state; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.state.KvState; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.junit.Test; + +import java.io.Serializable; + +import static org.junit.Assert.assertTrue; + + +public class StateBackendITCase extends StreamingMultipleProgramsTestBase { + + /** +* Verify that the user-specified state backend is used even if checkpointing is disabled. +* +* @throws Exception
[4/5] flink git commit: [FLINK-3292]Fix for Bug in flink-jdbc. Not all JDBC drivers supported
[FLINK-3292]Fix for Bug in flink-jdbc. Not all JDBC drivers supported This closes #1551 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d97fcda6 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d97fcda6 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d97fcda6 Branch: refs/heads/master Commit: d97fcda6635b6821ac3f61c39e0fa156bc7c7fd4 Parents: 83b88c2 Author: Subhobrata DeyAuthored: Wed Jan 27 17:00:37 2016 -0500 Committer: Stephan Ewen Committed: Thu Jan 28 14:37:53 2016 +0100 -- .../flink/api/java/io/jdbc/JDBCInputFormat.java | 16 +++- .../flink/api/java/io/jdbc/JDBCInputFormatTest.java | 3 +++ .../api/java/io/jdbc/JDBCOutputFormatTest.java | 3 +++ 3 files changed, 21 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/d97fcda6/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java -- diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java index eb3ac31..84eb309 100644 --- a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java +++ b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java @@ -58,6 +58,8 @@ public class JDBCInputFormat extends RichInputFormat
[1/5] flink git commit: [FLINK-3178] Don't Emit In-Flight Windows When Closing Window Operator
Repository: flink Updated Branches: refs/heads/master 2e2330737 -> 6b01a8902 [FLINK-3178] Don't Emit In-Flight Windows When Closing Window Operator This closes #1542 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c4e5a55f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c4e5a55f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c4e5a55f Branch: refs/heads/master Commit: c4e5a55f027ed73ce557f10d5125a0b168832889 Parents: 2e23307 Author: Aljoscha KrettekAuthored: Mon Jan 18 13:25:03 2016 +0100 Committer: Stephan Ewen Committed: Thu Jan 28 14:19:17 2016 +0100 -- .../examples/windowing/SessionWindowing.java| 2 +- .../util/TopSpeedWindowingExampleData.java | 8 +- ...ractAlignedProcessingTimeWindowOperator.java | 5 - .../windowing/NonKeyedWindowOperator.java | 14 +-- .../operators/windowing/WindowOperator.java | 17 ++- .../api/complex/ComplexIntegrationTest.java | 3 +- ...AlignedProcessingTimeWindowOperatorTest.java | 110 ++- ...AlignedProcessingTimeWindowOperatorTest.java | 89 --- .../util/OneInputStreamOperatorTestHarness.java | 3 +- 9 files changed, 54 insertions(+), 197 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/c4e5a55f/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java -- diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java index 035727a..baa4af8 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java @@ -116,7 +116,7 @@ public class SessionWindowing { // Update the last seen event time lastSeenState.update(timestamp); - ctx.registerEventTimeTimer(lastSeen + sessionTimeout); + ctx.registerEventTimeTimer(timestamp + sessionTimeout); if (timeSinceLastEvent > sessionTimeout) { return TriggerResult.FIRE_AND_PURGE; http://git-wip-us.apache.org/repos/asf/flink/blob/c4e5a55f/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java -- diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java index bf63695..4718b8b 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java @@ -192,9 +192,7 @@ public class TopSpeedWindowingExampleData { "(1,95,1973.61115,1424952007664)\n" + "(0,100,1709.72229,1424952006663)\n" + "(0,100,1737.50007,1424952007664)\n" + - "(1,95,1973.61115,1424952007664)\n" + - "(0,100,1791.66674,1424952009664)\n" + - "(1,95,2211.8,1424952017668)\n"; + "(1,95,1973.61115,1424952007664)\n"; public static final String TOP_CASE_CLASS_SPEEDS = "CarEvent(0,55,15.277,1424951918630)\n" + @@ -267,9 +265,7 @@ public class TopSpeedWindowingExampleData { "CarEvent(1,95,1973.61115,1424952007664)\n" + "CarEvent(0,100,1709.72229,1424952006663)\n" + "CarEvent(0,100,1737.50007,1424952007664)\n" + - "CarEvent(1,95,1973.61115,1424952007664)\n" + - "CarEvent(0,100,1791.66674,1424952009664)\n" + -
flink git commit: [FLINK-3242] Adjust StateBackendITCase for 0.10 signatures of state backends
Repository: flink Updated Branches: refs/heads/release-0.10 2aeb6fac3 -> bef39f881 [FLINK-3242] Adjust StateBackendITCase for 0.10 signatures of state backends Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/bef39f88 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/bef39f88 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/bef39f88 Branch: refs/heads/release-0.10 Commit: bef39f881a3d25334d29711808757e8944642c7e Parents: 2aeb6fa Author: Stephan EwenAuthored: Thu Jan 28 16:58:12 2016 +0100 Committer: Stephan Ewen Committed: Thu Jan 28 17:12:38 2016 +0100 -- .../runtime/state/StateBackendITCase.java | 35 ++-- 1 file changed, 18 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/bef39f88/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java index cdfef85..12233ce 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java @@ -18,22 +18,23 @@ package org.apache.flink.streaming.runtime.state; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.state.KvState; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; + import org.junit.Test; import java.io.Serializable; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class StateBackendITCase extends StreamingMultipleProgramsTestBase { @@ -70,19 +71,22 @@ public class StateBackendITCase extends StreamingMultipleProgramsTestBase { } }) .print(); - - boolean caughtSuccess = false; + try { see.execute(); - } catch (JobExecutionException e) { - if (e.getCause() instanceof SuccessException) { - caughtSuccess = true; - } else { - throw e; + fail("This should throw a 'SuccessException'"); + } + catch (JobExecutionException e) { + Throwable cause = e.getCause(); + if (cause == null || !(cause.getCause() instanceof SuccessException)) { + e.printStackTrace(); + fail(e.getMessage()); } } - - assertTrue(caughtSuccess); + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } } @@ -90,7 +94,7 @@ public class StateBackendITCase extends StreamingMultipleProgramsTestBase { private static final long serialVersionUID = 1L; @Override - public void initializeForJob(Environment env) throws Exception { + public void initializeForJob(JobID id) throws Exception { throw new SuccessException(); } @@ -105,11 +109,8 @@ public class StateBackendITCase extends StreamingMultipleProgramsTestBase { } @Override - public KvState createKvState(String stateId, - String stateName, - TypeSerializer keySerializer, - TypeSerializer valueSerializer, - V defaultValue) throws Exception { + public KvState createKvState( + TypeSerializer keySerializer, TypeSerializer valueSerializer, V defaultValue) throws Exception { return null;
flink git commit: [hotfix] Remove 'ByteArrayInputView' and replace deserialization in TypeInformationSerializationSchema with more efficient reusable buffers.
Repository: flink Updated Branches: refs/heads/master 67b380d61 -> 92efcd34a [hotfix] Remove 'ByteArrayInputView' and replace deserialization in TypeInformationSerializationSchema with more efficient reusable buffers. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/92efcd34 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/92efcd34 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/92efcd34 Branch: refs/heads/master Commit: 92efcd34a5da2bccb07666f2c647974ea3e7c94f Parents: 67b380d Author: Stephan EwenAuthored: Mon Feb 1 14:39:24 2016 +0100 Committer: Stephan Ewen Committed: Mon Feb 1 17:29:02 2016 +0100 -- .../typeutils/runtime/ByteArrayInputView.java | 40 .../runtime/kryo/KryoClearedBufferTest.java | 8 +++- .../runtime/util/DataInputDeserializer.java | 48 .../runtime/util/DataOutputSerializer.java | 25 ++ ...eInformationKeyValueSerializationSchema.java | 44 +- .../connectors/kafka/KafkaConsumerTestBase.java | 17 +-- .../TypeInformationSerializationSchema.java | 14 -- .../TypeInformationSerializationSchemaTest.java | 2 +- 8 files changed, 109 insertions(+), 89 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/92efcd34/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ByteArrayInputView.java -- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ByteArrayInputView.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ByteArrayInputView.java deleted file mode 100644 index 48d6a3d..000 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ByteArrayInputView.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.api.java.typeutils.runtime; - -import org.apache.flink.core.memory.DataInputView; - -import java.io.ByteArrayInputStream; -import java.io.DataInputStream; -import java.io.IOException; - -public class ByteArrayInputView extends DataInputStream implements DataInputView { - - public ByteArrayInputView(byte[] data) { - super(new ByteArrayInputStream(data)); - } - - @Override - public void skipBytesToRead(int numBytes) throws IOException { - while (numBytes > 0) { - int skipped = skipBytes(numBytes); - numBytes -= skipped; - } - } -} http://git-wip-us.apache.org/repos/asf/flink/blob/92efcd34/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java -- diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java index ab2e45f..7572408 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java @@ -22,13 +22,16 @@ import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; + import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.java.typeutils.runtime.ByteArrayInputView; import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputView; + import org.junit.Assert; import org.junit.Test; +import java.io.ByteArrayInputStream; import java.io.EOFException; import java.io.IOException; import java.io.Serializable; @@ -69,7 +72,8 @@ public class KryoClearedBufferTest {
[13/38] flink git commit: [FLINK-3303] [core] Move all type utilities to flink-core
http://git-wip-us.apache.org/repos/asf/flink/blob/21a71586/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java -- diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java new file mode 100644 index 000..a196984 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java @@ -0,0 +1,79 @@ +/* + * 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.api.java.typeutils.runtime; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; + +import org.apache.flink.api.common.typeutils.SerializerTestInstance; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple; +import org.junit.Assert; + +public class TupleSerializerTestInstance extends SerializerTestInstance { + + public TupleSerializerTestInstance(TypeSerializer serializer, Class typeClass, int length, T[] testData) { + super(serializer, typeClass, length, testData); + } + + protected void deepEquals(String message, T shouldTuple, T isTuple) { + Assert.assertEquals(shouldTuple.getArity(), isTuple.getArity()); + + for (int i = 0; i < shouldTuple.getArity(); i++) { + Object should = shouldTuple.getField(i); + Object is = isTuple.getField(i); + + if (should.getClass().isArray()) { + if (should instanceof boolean[]) { + Assert.assertTrue(message, Arrays.equals((boolean[]) should, (boolean[]) is)); + } + else if (should instanceof byte[]) { + assertArrayEquals(message, (byte[]) should, (byte[]) is); + } + else if (should instanceof short[]) { + assertArrayEquals(message, (short[]) should, (short[]) is); + } + else if (should instanceof int[]) { + assertArrayEquals(message, (int[]) should, (int[]) is); + } + else if (should instanceof long[]) { + assertArrayEquals(message, (long[]) should, (long[]) is); + } + else if (should instanceof float[]) { + assertArrayEquals(message, (float[]) should, (float[]) is, 0.0f); + } + else if (should instanceof double[]) { + assertArrayEquals(message, (double[]) should, (double[]) is, 0.0); + } + else if (should instanceof char[]) { + assertArrayEquals(message, (char[]) should, (char[]) is); + } + else { + assertArrayEquals(message, (Object[]) should, (Object[]) is); + } + } + else { + assertEquals(message, should, is); + } + } + } +} http://git-wip-us.apache.org/repos/asf/flink/blob/21a71586/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java -- diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java
[01/38] flink git commit: [hotfix] Reduce the heavy sysout verbosity for certain tests
Repository: flink Updated Branches: refs/heads/master 360f02b1f -> 8fc7e7af2 [hotfix] Reduce the heavy sysout verbosity for certain tests Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6f042e78 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6f042e78 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6f042e78 Branch: refs/heads/master Commit: 6f042e7894be388fa8e400a08002584c10781e60 Parents: 21a7158 Author: Stephan EwenAuthored: Mon Feb 1 16:46:03 2016 +0100 Committer: Stephan Ewen Committed: Tue Feb 2 16:55:44 2016 +0100 -- .../jar/CheckpointedStreamingProgram.java | 10 -- .../flink/test/recovery/FastFailuresITCase.java | 18 +++--- 2 files changed, 15 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6f042e78/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java -- diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java index 47253da..cda5a7b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java @@ -27,8 +27,6 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; import java.lang.RuntimeException; -import java.net.URL; -import java.net.URLClassLoader; /** * A simple streaming program, which is using the state checkpointing of Flink. @@ -40,14 +38,6 @@ public class CheckpointedStreamingProgram { private static final int CHECKPOINT_INTERVALL = 100; public static void main(String[] args) throws Exception { - ClassLoader cl = ClassLoader.getSystemClassLoader(); - URL[] urls = ((URLClassLoader)cl).getURLs(); - - for(URL url: urls){ - System.out.println(url.getFile()); - } - System.out.println("CheckpointedStreamingProgram classpath: "); - final String jarFile = args[0]; final String host = args[1]; final int port = Integer.parseInt(args[2]); http://git-wip-us.apache.org/repos/asf/flink/blob/6f042e78/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java -- diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java index 0684fde..2a139c7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java @@ -21,11 +21,13 @@ package org.apache.flink.test.recovery; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.junit.Test; @@ -33,6 +35,7 @@ import java.util.concurrent.atomic.AtomicInteger; import static org.junit.Assert.fail; +@SuppressWarnings("serial") public class FastFailuresITCase { static final AtomicInteger FAILURES_SO_FAR = new AtomicInteger(); @@ -40,12 +43,21 @@ public class FastFailuresITCase { @Test public void testThis() { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); + + ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(config, false); + cluster.start(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( + "localhost", cluster.getLeaderRPCPort()); + +
[05/38] flink git commit: [FLINK-3303] [core] Move all type utilities to flink-core
http://git-wip-us.apache.org/repos/asf/flink/blob/21a71586/flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java -- diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java deleted file mode 100644 index cd405bc..000 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * 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.api.java.typeutils; - -import static org.junit.Assert.*; - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple4; -import org.junit.Test; - -public class CompositeTypeTest { - - private final TupleTypeInfo tupleTypeInfo = new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - - - private final TupleTypeInfo > inNestedTuple1 = new TupleTypeInfo >( - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO); - - private final TupleTypeInfo > inNestedTuple2 = new TupleTypeInfo >( - BasicTypeInfo.DOUBLE_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO); - - private final TupleTypeInfo nestedTypeInfo = new TupleTypeInfo , Integer, Tuple2 >>( - BasicTypeInfo.INT_TYPE_INFO, - inNestedTuple1, - BasicTypeInfo.INT_TYPE_INFO, - inNestedTuple2); - - private final TupleTypeInfo >> inNestedTuple3 = new TupleTypeInfo >>( - BasicTypeInfo.INT_TYPE_INFO, - new TupleTypeInfo >(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO)); - - private final TupleTypeInfo deepNestedTupleTypeInfo = new TupleTypeInfo >, Integer>>( - BasicTypeInfo.INT_TYPE_INFO, - inNestedTuple3, - BasicTypeInfo.INT_TYPE_INFO ); - - private final PojoTypeInfo pojoTypeInfo = ((PojoTypeInfo) TypeExtractor.getForClass - (MyPojo.class)); - - private final TupleTypeInfo pojoInTupleTypeInfo = new TupleTypeInfo >(BasicTypeInfo.INT_TYPE_INFO, pojoTypeInfo); - - @Test - public void testGetFlatFields() { - assertEquals(0, tupleTypeInfo.getFlatFields("0").get(0).getPosition()); - assertEquals(1, tupleTypeInfo.getFlatFields("1").get(0).getPosition()); - assertEquals(2, tupleTypeInfo.getFlatFields("2").get(0).getPosition()); - assertEquals(3, tupleTypeInfo.getFlatFields("3").get(0).getPosition()); - assertEquals(0, tupleTypeInfo.getFlatFields("f0").get(0).getPosition()); - assertEquals(1, tupleTypeInfo.getFlatFields("f1").get(0).getPosition()); - assertEquals(2, tupleTypeInfo.getFlatFields("f2").get(0).getPosition()); - assertEquals(3, tupleTypeInfo.getFlatFields("f3").get(0).getPosition()); - - assertEquals(0, nestedTypeInfo.getFlatFields("0").get(0).getPosition()); - assertEquals(1, nestedTypeInfo.getFlatFields("1.0").get(0).getPosition()); - assertEquals(2, nestedTypeInfo.getFlatFields("1.1").get(0).getPosition()); - assertEquals(3, nestedTypeInfo.getFlatFields("1.2").get(0).getPosition()); - assertEquals(4, nestedTypeInfo.getFlatFields("2").get(0).getPosition()); -
[30/38] flink git commit: [FLINK-3303] [core] Move Tuple classes to flink-core
http://git-wip-us.apache.org/repos/asf/flink/blob/7081836e/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple20.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple20.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple20.java new file mode 100644 index 000..8116121 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple20.java @@ -0,0 +1,439 @@ +/* + * 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. + */ + + +// -- +// THIS IS A GENERATED SOURCE FILE. DO NOT EDIT! +// GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator. +// -- + + +package org.apache.flink.api.java.tuple; + +import org.apache.flink.util.StringUtils; + +/** + * A tuple with 20 fields. Tuples are strongly typed; each field may be of a separate type. + * The fields of the tuple can be accessed directly as public fields (f0, f1, ...) or via their position + * through the {@link #getField(int)} method. The tuple field positions start at zero. + * + * Tuples are mutable types, meaning that their fields can be re-assigned. This allows functions that work + * with Tuples to reuse objects in order to reduce pressure on the garbage collector. + * + * @see Tuple + * + * @param The type of field 0 + * @param The type of field 1 + * @param The type of field 2 + * @param The type of field 3 + * @param The type of field 4 + * @param The type of field 5 + * @param The type of field 6 + * @param The type of field 7 + * @param The type of field 8 + * @param The type of field 9 + * @param The type of field 10 + * @param The type of field 11 + * @param The type of field 12 + * @param The type of field 13 + * @param The type of field 14 + * @param The type of field 15 + * @param The type of field 16 + * @param The type of field 17 + * @param The type of field 18 + * @param The type of field 19 + */ +public class Tuple20extends Tuple { + + private static final long serialVersionUID = 1L; + + /** Field 0 of the tuple. */ + public T0 f0; + /** Field 1 of the tuple. */ + public T1 f1; + /** Field 2 of the tuple. */ + public T2 f2; + /** Field 3 of the tuple. */ + public T3 f3; + /** Field 4 of the tuple. */ + public T4 f4; + /** Field 5 of the tuple. */ + public T5 f5; + /** Field 6 of the tuple. */ + public T6 f6; + /** Field 7 of the tuple. */ + public T7 f7; + /** Field 8 of the tuple. */ + public T8 f8; + /** Field 9 of the tuple. */ + public T9 f9; + /** Field 10 of the tuple. */ + public T10 f10; + /** Field 11 of the tuple. */ + public T11 f11; + /** Field 12 of the tuple. */ + public T12 f12; + /** Field 13 of the tuple. */ + public T13 f13; + /** Field 14 of the tuple. */ + public T14 f14; + /** Field 15 of the tuple. */ + public T15 f15; + /** Field 16 of the tuple. */ + public T16 f16; + /** Field 17 of the tuple. */ + public T17 f17; + /** Field 18 of the tuple. */ + public T18 f18; + /** Field 19 of the tuple. */ + public T19 f19; + + /** +* Creates a new tuple where all fields are null. +*/ + public Tuple20() {} + + /** +* Creates a new tuple and assigns the given values to the tuple's fields. +* +* @param value0 The value for field 0 +* @param value1 The value for field 1 +* @param value2 The value for field 2 +* @param value3 The value for field 3 +* @param value4 The value for field 4 +* @param value5 The value for field 5 +* @param value6 The value for field 6 +* @param value7 The value for field 7 +* @param value8 The value for field 8 +* @param value9 The value for field 9 +* @param value10 The value for field 10 +
[08/38] flink git commit: [FLINK-3303] [core] Move all type utilities to flink-core
http://git-wip-us.apache.org/repos/asf/flink/blob/21a71586/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java -- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java deleted file mode 100644 index de24956..000 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java +++ /dev/null @@ -1,592 +0,0 @@ -/* - * 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.api.java.typeutils.runtime; - -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.lang.reflect.Field; -import java.lang.reflect.Modifier; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -import com.google.common.base.Preconditions; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - - -public final class PojoSerializer extends TypeSerializer { - - // Flags for the header - private static byte IS_NULL = 1; - private static byte NO_SUBCLASS = 2; - private static byte IS_SUBCLASS = 4; - private static byte IS_TAGGED_SUBCLASS = 8; - - private static final long serialVersionUID = 1L; - - private final Class clazz; - - private final TypeSerializer[] fieldSerializers; - - private final int numFields; - - private final MapregisteredClasses; - - private final TypeSerializer[] registeredSerializers; - - private final ExecutionConfig executionConfig; - - private transient Map subclassSerializerCache; - private transient ClassLoader cl; - // We need to handle these ourselves in writeObject()/readObject() - private transient Field[] fields; - - @SuppressWarnings("unchecked") - public PojoSerializer( - Class clazz, - TypeSerializer[] fieldSerializers, - Field[] fields, - ExecutionConfig executionConfig) { - - this.clazz = Preconditions.checkNotNull(clazz); - this.fieldSerializers = (TypeSerializer[]) Preconditions.checkNotNull(fieldSerializers); - this.fields = Preconditions.checkNotNull(fields); - this.numFields = fieldSerializers.length; - this.executionConfig = Preconditions.checkNotNull(executionConfig); - - LinkedHashSet registeredPojoTypes = executionConfig.getRegisteredPojoTypes(); - - for (int i = 0; i < numFields; i++) { - this.fields[i].setAccessible(true); - } - - cl = Thread.currentThread().getContextClassLoader(); - - subclassSerializerCache = new HashMap (); - - // We only want those classes that are not our own class and are actually sub-classes. - List cleanedTaggedClasses = new ArrayList (registeredPojoTypes.size()); - for (Class registeredClass: registeredPojoTypes) { - if (registeredClass.equals(clazz)) { - continue; - } - if (!clazz.isAssignableFrom(registeredClass)) { - continue; - } - cleanedTaggedClasses.add(registeredClass); - - } - this.registeredClasses = new LinkedHashMap (cleanedTaggedClasses.size()); - registeredSerializers = new
[23/38] flink git commit: [FLINK-3303] [core] Move Tuple classes to flink-core
http://git-wip-us.apache.org/repos/asf/flink/blob/7081836e/flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple15Builder.java -- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple15Builder.java b/flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple15Builder.java deleted file mode 100644 index b423a3b..000 --- a/flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple15Builder.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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. - */ - - -// -- -// THIS IS A GENERATED SOURCE FILE. DO NOT EDIT! -// GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator. -// -- - - -package org.apache.flink.api.java.tuple.builder; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.api.java.tuple.Tuple15; - -public class Tuple15Builder{ - - private List > tuples = new ArrayList >(); - - public Tuple15Builder add(T0 value0, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8, T9 value9, T10 value10, T11 value11, T12 value12, T13 value13, T14 value14){ - tuples.add(new Tuple15 (value0, value1, value2, value3, value4, value5, value6, value7, value8, value9, value10, value11, value12, value13, value14)); - return this; - } - - @SuppressWarnings("unchecked") - public Tuple15 [] build(){ - return tuples.toArray(new Tuple15[tuples.size()]); - } -} http://git-wip-us.apache.org/repos/asf/flink/blob/7081836e/flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple16Builder.java -- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple16Builder.java b/flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple16Builder.java deleted file mode 100644 index c698730..000 --- a/flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple16Builder.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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. - */ - - -// -- -// THIS IS A GENERATED SOURCE FILE. DO NOT EDIT! -// GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator. -// -- - - -package org.apache.flink.api.java.tuple.builder; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.api.java.tuple.Tuple16; - -public class Tuple16Builder { - - private List > tuples = new ArrayList
[04/38] flink git commit: [FLINK-3303] [core] Move all type utilities to flink-core
http://git-wip-us.apache.org/repos/asf/flink/blob/21a71586/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeComparatorTest.java -- diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeComparatorTest.java deleted file mode 100644 index 19fac43..000 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeComparatorTest.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.api.java.typeutils.runtime; - -import org.apache.flink.api.common.typeutils.TypeSerializer; - -public class AvroGenericTypeComparatorTest extends AbstractGenericTypeComparatorTest { - @Override - protected TypeSerializer createSerializer(Class type) { - return new AvroSerializer(type); - } -} http://git-wip-us.apache.org/repos/asf/flink/blob/21a71586/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java -- diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java deleted file mode 100644 index df1ff60..000 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.api.java.typeutils.runtime; - -import org.apache.flink.api.common.typeutils.TypeSerializer; - -public class AvroGenericTypeSerializerTest extends AbstractGenericTypeSerializerTest { - - @Override - protected TypeSerializer createSerializer(Class type) { - return new AvroSerializer(type); - } -} http://git-wip-us.apache.org/repos/asf/flink/blob/21a71586/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java -- diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java deleted file mode 100644 index 8a89410..000 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java +++ /dev/null @@ -1,189 +0,0 @@ -/* - * 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,