Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 0E9CA18599 for ; Thu, 22 Oct 2015 09:26:54 +0000 (UTC) Received: (qmail 64810 invoked by uid 500); 22 Oct 2015 09:26:53 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 64705 invoked by uid 500); 22 Oct 2015 09:26:53 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 64407 invoked by uid 99); 22 Oct 2015 09:26:53 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 22 Oct 2015 09:26:53 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 95545E13D0; Thu, 22 Oct 2015 09:26:53 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: vozerov@apache.org To: commits@ignite.apache.org Date: Thu, 22 Oct 2015 09:27:00 -0000 Message-Id: In-Reply-To: <9d4d1a7cfd0c49149f55244b156203a3@git.apache.org> References: <9d4d1a7cfd0c49149f55244b156203a3@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [08/50] [abbrv] ignite git commit: IGNITE-1682 .Net: Remove RemoteListen from Events API. IGNITE-1682 .Net: Remove RemoteListen from Events API. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/91eeab7a Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/91eeab7a Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/91eeab7a Branch: refs/heads/ignite-1651 Commit: 91eeab7ac3e29cdba4cdb9679a0c083aea8fc959 Parents: 6695e6c Author: Pavel Tupitsyn Authored: Thu Oct 15 15:04:21 2015 +0300 Committer: vozerov-gridgain Committed: Thu Oct 15 15:04:21 2015 +0300 ---------------------------------------------------------------------- .../Apache.Ignite.Core.Tests/EventsTest.cs | 38 ++++---- .../Apache.Ignite.Core.csproj | 1 + .../Apache.Ignite.Core/Events/IEventFilter.cs | 9 +- .../Apache.Ignite.Core/Events/IEventListener.cs | 34 +++++++ .../dotnet/Apache.Ignite.Core/Events/IEvents.cs | 97 ++------------------ .../Impl/Common/DelegateTypeDescriptor.cs | 8 +- .../Apache.Ignite.Core/Impl/Events/Events.cs | 37 +++++--- .../Impl/Events/RemoteListenEventFilter.cs | 10 +- .../Events/EventsExample.cs | 18 ---- .../Apache.Ignite.ExamplesDll.csproj | 1 - .../Events/LocalListener.cs | 5 +- .../Events/RemoteFilter.cs | 42 --------- 12 files changed, 102 insertions(+), 198 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs index b325d36..33841ad 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs @@ -348,7 +348,7 @@ namespace Apache.Ignite.Core.Tests // Filter waitTask = getWaitTask(() => events.WaitForLocal( - new EventFilter((g, e) => e.Type == EventType.TaskReduced))); + new EventFilter(e => e.Type == EventType.TaskReduced))); Assert.IsTrue(waitTask.Wait(timeout)); Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result); @@ -356,13 +356,14 @@ namespace Apache.Ignite.Core.Tests // Filter & types waitTask = getWaitTask(() => events.WaitForLocal( - new EventFilter((g, e) => e.Type == EventType.TaskReduced), EventType.TaskReduced)); + new EventFilter(e => e.Type == EventType.TaskReduced), EventType.TaskReduced)); Assert.IsTrue(waitTask.Wait(timeout)); Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result); Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type); } + /* /// /// Tests RemoteListen. /// @@ -426,7 +427,7 @@ namespace Apache.Ignite.Core.Tests CheckSend(1, typeof(JobEvent), expectedType); // one last event CheckNoEvent(); - } + }*/ /// /// Tests RemoteQuery. @@ -788,7 +789,7 @@ namespace Apache.Ignite.Core.Tests /// Gets the event listener. /// /// New instance of event listener. - public static IEventFilter GetListener() + public static IEventListener GetListener() { return new EventFilter(Listen); } @@ -813,13 +814,12 @@ namespace Apache.Ignite.Core.Tests /// /// Listen method. /// - /// Originating node ID. /// Event. - private static bool Listen(Guid? id, IEvent evt) + private static bool Listen(IEvent evt) { try { - LastNodeIds.Push(id); + LastNodeIds.Push(evt.Node.Id); ReceivedEvents.Push(evt); ReceivedEvent.Signal(); @@ -830,7 +830,7 @@ namespace Apache.Ignite.Core.Tests { // When executed on remote nodes, these exceptions will not go to sender, // so we have to accumulate them. - Failures.Push(string.Format("Exception in Listen (msg: {0}, id: {1}): {2}", evt, id, ex)); + Failures.Push(string.Format("Exception in Listen (msg: {0}, id: {1}): {2}", evt, evt.Node.Id, ex)); throw; } } @@ -840,28 +840,34 @@ namespace Apache.Ignite.Core.Tests /// Test event filter. /// [Serializable] - public class EventFilter : IEventFilter where T : IEvent + public class EventFilter : IEventFilter, IEventListener where T : IEvent { /** */ - private readonly Func _invoke; + private readonly Func _invoke; /// /// Initializes a new instance of the class. /// /// The invoke delegate. - public EventFilter(Func invoke) + public EventFilter(Func invoke) { _invoke = invoke; } /** */ - bool IEventFilter.Invoke(Guid? nodeId, T evt) + bool IEventFilter.Invoke(T evt) + { + return _invoke(evt); + } + + /** */ + bool IEventListener.Invoke(T evt) { - return _invoke(nodeId, evt); + return _invoke(evt); } /** */ - public bool Invoke(Guid nodeId, T evt) + public bool Invoke(T evt) { throw new Exception("Invalid method"); } @@ -882,7 +888,7 @@ namespace Apache.Ignite.Core.Tests } /** */ - public bool Invoke(Guid? nodeId, IEvent evt) + public bool Invoke(IEvent evt) { return evt.Type == _type; } @@ -906,7 +912,7 @@ namespace Apache.Ignite.Core.Tests } /** */ - public bool Invoke(Guid? nodeId, IEvent evt) + public bool Invoke(IEvent evt) { return evt.Type == _type; } http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index a10a0a5..401b46c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -125,6 +125,7 @@ + http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventFilter.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventFilter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventFilter.cs index 83aca53..8c80cec 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventFilter.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventFilter.cs @@ -17,8 +17,6 @@ namespace Apache.Ignite.Core.Events { - using System; - /// /// Represents an event filter. /// @@ -26,11 +24,10 @@ namespace Apache.Ignite.Core.Events public interface IEventFilter where T : IEvent { /// - /// Determines whether specified event passes this filtger. + /// Determines whether specified event passes this filter. /// - /// Node identifier. /// Event. - /// Value indicating whether specified event passes this filtger. - bool Invoke(Guid? nodeId, T evt); + /// Value indicating whether specified event passes this filter. + bool Invoke(T evt); } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventListener.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventListener.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventListener.cs new file mode 100644 index 0000000..06e9ecc --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventListener.cs @@ -0,0 +1,34 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Events +{ + /// + /// Represents an event listener. + /// + /// Event type. + public interface IEventListener where T : IEvent + { + /// + /// Invoked when event occurs. + /// + /// Event. + /// Value indicating whether this listener should be kept subscribed. + /// Returning false unsubscribes this listener from future notifications. + bool Invoke(T evt); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEvents.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEvents.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEvents.cs index b2f07d4..e8459c6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEvents.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEvents.cs @@ -23,7 +23,7 @@ namespace Apache.Ignite.Core.Events using Apache.Ignite.Core.Common; /// - /// Provides functionality for local and remote event notifications on nodes defined by . + /// Provides functionality for event notifications on nodes defined by . /// /// All members are thread-safe and may be used concurrently from multiple threads. /// @@ -59,91 +59,6 @@ namespace Apache.Ignite.Core.Events where T : IEvent; /// - /// Adds event listener for specified events to all nodes in the cluster group (possibly including local node - /// if it belongs to the cluster group as well). This means that all events occurring on any node within this - /// cluster group that pass remote filter will be sent to local node for local listener notifications. - /// - /// The listener can be unsubscribed automatically if local node stops, if localListener callback - /// returns false or if is called. - /// - /// Type of events. - /// Remote events buffer size. Events from remote nodes won't be sent until buffer - /// is full or time interval is exceeded. - /// Maximum time interval after which events from remote node will be sent. Events - /// from remote nodes won't be sent until buffer is full or time interval is exceeded. - /// Flag indicating that event listeners on remote nodes should be automatically - /// unregistered if master node (node that initiated event listening) leaves topology. - /// If this flag is false, listeners will be unregistered only when - /// method is called, or the localListener returns false. - /// Listener callback that is called on local node. If null, these events will - /// be handled on remote nodes by passed in remoteFilter. - /// - /// Filter callback that is called on remote node. Only events that pass the remote filter will be - /// sent to local node. If null, all events of specified types will be sent to local node. - /// This remote filter can be used to pre-handle events remotely, before they are passed in to local callback. - /// It will be auto-unsubscribed on the node where event occurred in case if it returns false. - /// - /// - /// Types of events to listen for. If not provided, all events that pass the provided remote filter - /// will be sent to local node. - /// - /// - /// Operation ID that can be passed to method to stop listening. - /// - [AsyncSupported] - Guid? RemoteListen(int bufSize = 1, TimeSpan? interval = null, bool autoUnsubscribe = true, - IEventFilter localListener = null, IEventFilter remoteFilter = null, params int[] types) - where T : IEvent; - - /// - /// Adds event listener for specified events to all nodes in the cluster group (possibly including local node - /// if it belongs to the cluster group as well). This means that all events occurring on any node within this - /// cluster group that pass remote filter will be sent to local node for local listener notifications. - /// - /// The listener can be unsubscribed automatically if local node stops, if localListener callback - /// returns false or if is called. - /// - /// Type of events. - /// Remote events buffer size. Events from remote nodes won't be sent until buffer - /// is full or time interval is exceeded. - /// Maximum time interval after which events from remote node will be sent. Events - /// from remote nodes won't be sent until buffer is full or time interval is exceeded. - /// Flag indicating that event listeners on remote nodes should be automatically - /// unregistered if master node (node that initiated event listening) leaves topology. - /// If this flag is false, listeners will be unregistered only when - /// method is called, or the localListener returns false. - /// Listener callback that is called on local node. If null, these events will - /// be handled on remote nodes by passed in remoteFilter. - /// - /// Filter callback that is called on remote node. Only events that pass the remote filter will be - /// sent to local node. If null, all events of specified types will be sent to local node. - /// This remote filter can be used to pre-handle events remotely, before they are passed in to local callback. - /// It will be auto-unsubscribed on the node where event occurred in case if it returns false. - /// - /// - /// Types of events to listen for. If not provided, all events that pass the provided remote filter - /// will be sent to local node. - /// - /// - /// Operation ID that can be passed to method to stop listening. - /// - [AsyncSupported] - Guid? RemoteListen(int bufSize = 1, TimeSpan? interval = null, bool autoUnsubscribe = true, - IEventFilter localListener = null, IEventFilter remoteFilter = null, IEnumerable types = null) - where T : IEvent; - - /// - /// Stops listening to remote events. This will unregister all listeners identified with provided operation ID - /// on all nodes defined by . - /// - /// - /// Operation ID that was returned from - /// . - /// - [AsyncSupported] - void StopRemoteListen(Guid opId); - - /// /// Waits for the specified events. /// /// Types of the events to wait for. @@ -205,7 +120,7 @@ namespace Apache.Ignite.Core.Events /// Attempt to record internal event with this method will cause to be thrown. /// /// Locally generated event. - /// If event type is within Ignite reserved range (1 � 1000) + /// If event type is within Ignite reserved range (1 to 1000) void RecordLocal(IEvent evt); /// @@ -216,7 +131,7 @@ namespace Apache.Ignite.Core.Events /// Predicate that is called on each received event. If predicate returns false, /// it will be unregistered and will stop receiving events. /// Event types for which this listener will be notified, should not be empty. - void LocalListen(IEventFilter listener, params int[] types) where T : IEvent; + void LocalListen(IEventListener listener, params int[] types) where T : IEvent; /// /// Adds an event listener for local events. Note that listener will be added regardless of whether @@ -226,7 +141,7 @@ namespace Apache.Ignite.Core.Events /// Predicate that is called on each received event. If predicate returns false, /// it will be unregistered and will stop receiving events. /// Event types for which this listener will be notified, should not be empty. - void LocalListen(IEventFilter listener, IEnumerable types) where T : IEvent; + void LocalListen(IEventListener listener, IEnumerable types) where T : IEvent; /// /// Removes local event listener. @@ -236,7 +151,7 @@ namespace Apache.Ignite.Core.Events /// Types of events for which to remove listener. If not specified, then listener /// will be removed for all types it was registered for. /// True if listener was removed, false otherwise. - bool StopLocalListen(IEventFilter listener, params int[] types) where T : IEvent; + bool StopLocalListen(IEventListener listener, params int[] types) where T : IEvent; /// /// Removes local event listener. @@ -246,7 +161,7 @@ namespace Apache.Ignite.Core.Events /// Types of events for which to remove listener. If not specified, then listener /// will be removed for all types it was registered for. /// True if listener was removed, false otherwise. - bool StopLocalListen(IEventFilter listener, IEnumerable types) where T : IEvent; + bool StopLocalListen(IEventListener listener, IEnumerable types) where T : IEvent; /// /// Enables provided events. Allows to start recording events that were disabled before. http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs index 0f2b3c1..fb55d8e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs @@ -48,7 +48,7 @@ namespace Apache.Ignite.Core.Impl.Common private readonly Func _computeFunc; /** */ - private readonly Func _eventFilter; + private readonly Func _eventFilter; /** */ private readonly Func _cacheEntryFilter; @@ -100,7 +100,7 @@ namespace Apache.Ignite.Core.Impl.Common /// /// Type. /// Precompiled invocator delegate. - public static Func GetEventFilter(Type type) + public static Func GetEventFilter(Type type) { return Get(type)._eventFilter; } @@ -245,8 +245,8 @@ namespace Apache.Ignite.Core.Impl.Common var args = iface.GetGenericArguments(); - _eventFilter = DelegateConverter.CompileFunc>(iface, - new[] {typeof (Guid?), args[0]}, new[] {false, true, false}); + _eventFilter = DelegateConverter.CompileFunc>(iface, + new[] {args[0]}, new[] {true, false}); } else if (genericTypeDefinition == typeof (ICacheEntryFilter<,>)) { http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs index 08936e4..6898a58 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs @@ -148,7 +148,7 @@ namespace Apache.Ignite.Core.Impl.Events if (localListener != null) { - var listener = new RemoteListenEventFilter(Ignite, (id, e) => localListener.Invoke(id, (T) e)); + var listener = new RemoteListenEventFilter(Ignite, e => localListener.Invoke((T) e)); writer.WriteLong(Ignite.HandleRegistry.Allocate(listener)); } @@ -230,11 +230,11 @@ namespace Apache.Ignite.Core.Impl.Events /** */ public void RecordLocal(IEvent evt) { - throw new NotImplementedException("GG-10244"); + throw new NotImplementedException("IGNITE-1410"); } /** */ - public void LocalListen(IEventFilter listener, params int[] types) where T : IEvent + public void LocalListen(IEventListener listener, params int[] types) where T : IEvent { IgniteArgumentCheck.NotNull(listener, "listener"); IgniteArgumentCheck.NotNullOrEmpty(types, "types"); @@ -244,13 +244,13 @@ namespace Apache.Ignite.Core.Impl.Events } /** */ - public void LocalListen(IEventFilter listener, IEnumerable types) where T : IEvent + public void LocalListen(IEventListener listener, IEnumerable types) where T : IEvent { LocalListen(listener, TypesToArray(types)); } /** */ - public bool StopLocalListen(IEventFilter listener, params int[] types) where T : IEvent + public bool StopLocalListen(IEventListener listener, params int[] types) where T : IEvent { lock (_localFilters) { @@ -271,7 +271,7 @@ namespace Apache.Ignite.Core.Impl.Events } /** */ - public bool StopLocalListen(IEventFilter listener, IEnumerable types) where T : IEvent + public bool StopLocalListen(IEventListener listener, IEnumerable types) where T : IEvent { return StopLocalListen(listener, TypesToArray(types)); } @@ -415,7 +415,7 @@ namespace Apache.Ignite.Core.Impl.Events /// Type of events. /// Predicate that is called on each received event. /// Event type for which this listener will be notified - private void LocalListen(IEventFilter listener, int type) where T : IEvent + private void LocalListen(IEventListener listener, int type) where T : IEvent { lock (_localFilters) { @@ -432,7 +432,7 @@ namespace Apache.Ignite.Core.Impl.Events if (!filters.TryGetValue(type, out localFilter)) { - localFilter = CreateLocalFilter(listener, type); + localFilter = CreateLocalListener(listener, type); filters[type] = localFilter; } @@ -448,10 +448,10 @@ namespace Apache.Ignite.Core.Impl.Events /// Listener. /// Event type. /// Created wrapper. - private LocalHandledEventFilter CreateLocalFilter(IEventFilter listener, int type) where T : IEvent + private LocalHandledEventFilter CreateLocalListener(IEventListener listener, int type) where T : IEvent { var result = new LocalHandledEventFilter( - stream => InvokeLocalFilter(stream, listener), + stream => InvokeLocalListener(stream, listener), unused => { lock (_localFilters) @@ -484,8 +484,21 @@ namespace Apache.Ignite.Core.Impl.Events { var evt = EventReader.Read(Marshaller.StartUnmarshal(stream)); - // No guid in local mode - return listener.Invoke(Guid.Empty, evt); + return listener.Invoke(evt); + } + + /// + /// Invokes local filter using data from specified stream. + /// + /// Event object type. + /// The stream. + /// The listener. + /// Filter invocation result. + private bool InvokeLocalListener(IPortableStream stream, IEventListener listener) where T : IEvent + { + var evt = EventReader.Read(Marshaller.StartUnmarshal(stream)); + + return listener.Invoke(evt); } /// http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs index ac50b35..f8e2f95 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs @@ -33,14 +33,14 @@ namespace Apache.Ignite.Core.Impl.Events private readonly Ignite _ignite; /** */ - private readonly Func _filter; + private readonly Func _filter; /// /// Initializes a new instance of the class. /// /// The grid. /// The filter. - public RemoteListenEventFilter(Ignite ignite, Func filter) + public RemoteListenEventFilter(Ignite ignite, Func filter) { _ignite = ignite; _filter = filter; @@ -53,9 +53,9 @@ namespace Apache.Ignite.Core.Impl.Events var evt = EventReader.Read(reader); - var nodeId = reader.ReadGuid(); + reader.ReadGuid(); // unused node id - return _filter(nodeId, evt) ? 1 : 0; + return _filter(evt) ? 1 : 0; } /// @@ -78,7 +78,7 @@ namespace Apache.Ignite.Core.Impl.Events var func = DelegateTypeDescriptor.GetEventFilter(pred.GetType()); - return new RemoteListenEventFilter(grid, (id, evt) => func(pred, id, evt)); + return new RemoteListenEventFilter(grid, evt => func(pred, evt)); } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs index c1f3035..f9d54b9 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs @@ -71,24 +71,6 @@ namespace Apache.Ignite.Examples.Events Console.WriteLine(">>> Received events count: " + listener.EventsReceived); Console.WriteLine(); - - // Remote listen example (start standalone nodes for better demonstration) - Console.WriteLine(">>> Listening for remote events..."); - - var localListener = new LocalListener(); - var remoteFilter = new RemoteFilter(); - - var listenId = ignite.GetEvents().RemoteListen(localListener: localListener, - remoteFilter: remoteFilter, types: EventType.JobExecutionAll); - - if (listenId == null) - throw new InvalidOperationException("Subscription failed."); - - ExecuteTask(ignite); - - ignite.GetEvents().StopRemoteListen(listenId.Value); - - Console.WriteLine(">>> Received events count: " + localListener.EventsReceived); } Console.WriteLine(); http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj index d579a77..441e4e0 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj @@ -49,7 +49,6 @@ - http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs index 067bd2a..8c689dc 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs @@ -24,7 +24,7 @@ namespace Apache.Ignite.ExamplesDll.Events /// /// Local event listener. /// - public class LocalListener : IEventFilter + public class LocalListener : IEventListener { /** Сount of received events. */ private int _eventsReceived; @@ -40,10 +40,9 @@ namespace Apache.Ignite.ExamplesDll.Events /// /// Determines whether specified event passes this filter. /// - /// Node identifier. /// Event. /// Value indicating whether specified event passes this filter. - public bool Invoke(Guid? nodeId, IEvent evt) + public bool Invoke(IEvent evt) { Interlocked.Increment(ref _eventsReceived); http://git-wip-us.apache.org/repos/asf/ignite/blob/91eeab7a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/RemoteFilter.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/RemoteFilter.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/RemoteFilter.cs deleted file mode 100644 index 45a957c..0000000 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/RemoteFilter.cs +++ /dev/null @@ -1,42 +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. - */ - -using System; -using Apache.Ignite.Core.Events; - -namespace Apache.Ignite.ExamplesDll.Events -{ - /// - /// Remote event filter. - /// - [Serializable] - public class RemoteFilter : IEventFilter - { - /// - /// Determines whether specified event passes this filter. - /// - /// Node identifier. - /// Event. - /// Value indicating whether specified event passes this filter. - public bool Invoke(Guid? nodeId, IEvent evt) - { - Console.WriteLine("Remote filter received event [evt={0}]", evt.Name); - - return evt is JobEvent; - } - } -} \ No newline at end of file