You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@geode.apache.org by Barry Oglesby <bo...@pivotal.io> on 2017/12/08 23:16:33 UTC

Re: [discussion]clear method implementation for Parallel Gateway Sender Queue

Dinesh,

I'm not sure if its better to send these comments on the dev list or on the
PR.

In any event, I'm not sure that ParallelGatewaySenderQueue should be
implementing PartitionedRegion clear.

I think PartitionedRegion clear should be implemented first, then
GatewaySender clearQueue should be built on top of that. It would use
PartitionedRegion clear and override BucketRegion clear as necessary in
AbstractBucketRegionQueue.

Your implementation of ParallelGatewaySenderQueue clearPartitionedRegion
(or some form of it) would become PartitionedRegion clear, although someone
who knows storage would have to review that to verify.

PartitionedRegion clear would delegate to BucketRegion clear which would be
like your ParallelGatewaySenderQueue clearBucketRegion. There might be a
better way to clear the BucketRegion than iterating the keys and destroying
each, but I'm not sure. Again someone who knows storage would have to
review that to verify.

Then, AbstractGatewaySender clearQueue could be implemented on top of that
something like:

- get RegionQueues
- for each RegionQueue, get its region or regions (depending on whether the
sender is serial or parallel)
- invoke clear on each region (or possibly localClear)

I think the implementation of PartitionedRegion clear and the testing of it
is trickier than it sounds. Concurrent updates, HA, rebalancing, etc. all
have to be taken into account.


Thanks,
Barry Oglesby


On Thu, Dec 7, 2017 at 2:21 AM, Dinesh Akhand <di...@amdocs.com> wrote:

> Created the pull request for the same.
> https://github.com/apache/geode/pull/1139
>
>
> Thanks,
> Dinesh akhand
>
> -----Original Message-----
> From: Dan Smith [mailto:dsmith@pivotal.io]
> Sent: Tuesday, July 18, 2017 3:59 AM
> To: dev@geode.apache.org
> Subject: Re: need information about SerialGatewaySenderQueue/ParallelGatewaySenderQueue
> Clear
>
> Hi Dinesh,
>
> I think we probably just never got around to adding a clear. I think you
> could probably clear your queues just stop stopping and starting the
> gateway sender, which might be the easiest thing to do here.
>
> Regarding your code, for your parallel queue are you doing that inside of
> a function? The code you have will try to clear things on a single node.
> The queue also maintains some other metadata in memory. I'm not quite sure
> what the effect on the queue will be if you delete the region entries
> without changing that other metadata. I guess you could test it and find
> out.
> You'll probably want to see what the effect is while the queue is actually
> dispatching entries as well, because it's possible you could catch the
> system in a state where it is trying to read entries from the region as you
> are deleting them. Or maybe pause the queue first in your clear method?
>
> -Dan
>
> On Fri, Jul 14, 2017 at 2:23 AM, Dinesh Akhand <di...@amdocs.com>
> wrote:
>
> > Hi Team,
> >
> >
> >
> > Please reply . why we don't have implementation of clear method in
> > ParallelGatewaySenderQueue/ SerialGatewaySenderQueue in geode.
> Requirement:
> > we want to clear the queue data.
> >
> >
> >
> > I have implement below method in our code.
> >
> > --------------------------------------------------------------
> >
> > Class ParallelGatewaySenderQueue.java
> >
> >
> >
> > //clear the partition region
> >
> > private void clearPartitionedRegion(PartitionedRegion
> > partitionedRegion)
> >
> > {
> >
> >                     LocalDataSet lds = (LocalDataSet)
> > PartitionRegionHelper.getLocalPrimaryData(partitionedRegion);
> >
> >                     Set<Integer>set = lds.getBucketSet(); // this
> > returns bucket ids in the function context
> >
> >
> >
> >                     for (Integer bucketId : set) {
> >
> >                                         Bucket bucket =
> partitionedRegion.
> > getRegionAdvisor().getBucket(bucketId);
> >
> >                                         if (bucket instanceof
> > ProxyBucketRegion == false) {
> >
> >                                                             if (bucket
> > instanceof BucketRegion) {
> >
> >
> >       BucketRegion bucketRegion = (BucketRegion) bucket;
> >
> >
> >       Set keySet = bucketRegion.keySet();
> >
> >
> >       for (Iterator iterator = keySet.iterator(); iterator.hasNext();)
> > {
> >
> >
> >                           Object key = iterator.next();
> >
> >
> >                           bucketRegion.remove(key);
> >
> >
> >       }
> >
> >                                                             }
> >
> >                                         }
> >
> >                     }
> >
> > }
> >
> > -------------------------------------------------------------
> >
> > Class : SerialGatewaySenderQueue.java
> >
> >  @Override
> >
> >   public void clearQueue() {
> >
> >
> >
> >     this.sender.getLifeCycleLock().readLock().lock();
> >
> >     Set<Long> keys = this.region.keys();
> >
> >     for (Long key : keys) {
> >
> >       this.region.remove(key);
> >
> >     }
> >
> >     this.sender.getLifeCycleLock().readLock().unlock();
> >
> >
> >
> >   }
> >
> > -----------------------------------------------------------------
> >
> >
> >
> > Any comment in above code will welcome.
> >
> >
> >
> >
> >
> > Thanks,
> >
> > Dinesh Akhand
> >
> >
> >
> > -----Original Message-----
> > From: Dinesh Akhand
> > Sent: Monday, May 15, 2017 2:39 PM
> > To: dev@geode.apache.org
> > Subject: need information about RegionQueue
> >
> >
> >
> >
> >
> > Hi Team,
> >
> >
> >
> > Why we do't have support to clear complete queue.  Is there any
> > limitation for it?.
> >
> >
> >
> > public void clear(PartitionedRegion pr, int bucketId) {
> >
> >        throw new RuntimeException("This method(clear)is not supported
> > by ParallelGatewaySenderQueue");
> >
> >   }
> >
> >
> >
> > Class : ParallelGatewaySenderQueue
> >
> > Class : SerialGatewaySenderQueue
> >
> >
> >
> > Thanks,
> >
> > Dinesh Akhand
> >
> >
> >
> > This message and the information contained herein is proprietary and
> > confidential and subject to the Amdocs policy statement,
> >
> >
> >
> > you may review at https://www.amdocs.com/about/email-disclaimer <
> > https://www.amdocs.com/about/email-disclaimer>
> > This message and the information contained herein is proprietary and
> > confidential and subject to the Amdocs policy statement,
> >
> > you may review at https://www.amdocs.com/about/email-disclaimer <
> > https://www.amdocs.com/about/email-disclaimer>
> >
> This message and the information contained herein is proprietary and
> confidential and subject to the Amdocs policy statement,
>
> you may review at https://www.amdocs.com/about/email-disclaimer <
> https://www.amdocs.com/about/email-disclaimer>
>