Something went wrong with gmail formatting. Resending my reply.

Alex,

Also shutdown policy must be always consistent on the grid or unintentional
> data loss is possible if two nodes are stopping simultaneously with
> different policies.

 Totally agree.

Let's use shutdownPolicy=DEFAULT|GRACEFUL, as was proposed by me earlier.

 I'm ok with GRACEFUL instead of WAIT_FOR_BACKUPS.

5. Let's keep a static property for simplifying setting of initial
> behavior.
> In most cases the policy will never be changed during grid's lifetime.
> No need for an explicit call to API on grid start.
> A joining node should check a local configuration value to match the grid.
> If a dynamic value is already present in a metastore, it should override
> static value with a warning.

To sum it up:
- ShutdownPolicy can be set with static configuration
(IgniteConfiguration#setShutdownPolicy), on join we validate that
statically configured policies on different server nodes are the same
- It's possible to override statically configured value by adding
distributed metastorage value, which can be done by
calling ignite.cluster().setShutdownPolicy(plc) or control.sh method
- Dynamic property is persisted

Generally, I don't mind if we have both dynamic and static configuration
properties. Necessity to call ignite.cluster().setShutdownPolicy(plc); on
every new cluster creation is a usability issue itself.
What bothers me here are the possible conflicts between static and dynamic
configuration. User may be surprised if he has shutdown policy X in
IgniteConfiguration, but the cluster behaves according to policy Y (because
several months ago another admin had called
IgniteCluster#setShutdownPolicy).
We can handle it by adding a separate enum field to the shutdown policy:

> public enum ShutdownPolicy {
>   /* Default value of dynamic shutdown policy property. If it's set, the
> shutdown policy is resolved according to value of static {@link
> IgniteConfiguration#shutdownPolicy} configuration parameter. */
>   USE_STATIC_CONFIGURATION,
>
>   /* Node leaves the cluster even if it's the last owner of some
> partitions. Only partitions of caches with backups > 0 are taken into
> account. */
>   IMMEDIATE,
>
>   /* Shutdown is blocked until node is safe to leave without the data
> loss. */
>   GRACEFUL
> }
>
This way:
1) User may easily understand whether the static parameter is overridden by
dynamic. If ignite.cluster().getShutdownPolicy() return anything except
USE_STATIC_CONFIGURATION, behavior is overridden.
2) User may clear previous overriding by calling
ignite.cluster().setShutdownPolicy(USE_STATIC_CONFIGURATION). After that,
behavior will be resolved based in IgniteConfiguration#shutdownPolicy again.
If we agree on this mechanism, I propose to use IMMEDIATE name instead of
DEFAULT for non-safe policy in order to don't confuse user.
Meanwhile, static configuration will accept the same enum, but
USE_STATIC_CONFIGURATION will be restricted:

> public class IgniteConfiguration {
>   public static final ShutdownPolicy DFLT_STATIC_SHUTDOWN_POLICY =
> IMMEDIATE;
>   private ShutdownPolicy shutdownPolicy = DFLT_STATIC_SHUTDOWN_POLICY;
>   ...
>   public void setShutdownPolicy(ShutdownPolicy shutdownPlc) {
>     if (shutdownPlc ==  USE_STATIC_CONFIGURATION)
>       throw new IllegalArgumentException("USE_STATIC_CONFIGURATION can
> only be passed as dynamic property value via
> ignite.cluster().setShutdownPolicy");
>     ...
>   }
> ...
> }
>

What do you think?

On Tue, Jun 9, 2020 at 3:09 PM Ivan Rakov <ivan.glu...@gmail.com> wrote:

> Alex,
>
> Also shutdown policy must be always consistent on the grid or unintentional
>> data loss is possible if two nodes are stopping simultaneously with
>> different policies.
>
>  Totally agree.
>
> Let's use shutdownPolicy=DEFAULT|GRACEFUL, as was proposed by me earlier.
>
>  I'm ok with GRACEFUL instead of WAIT_FOR_BACKUPS.
>
> 5. Let's keep a static property for simplifying setting of initial
>> behavior.
>> In most cases the policy will never be changed during grid's lifetime.
>> No need for an explicit call to API on grid start.
>> A joining node should check a local configuration value to match the grid.
>> If a dynamic value is already present in a metastore, it should override
>> static value with a warning.
>
> To sum it up:
> - ShutdownPolicy can be set with static configuration
> (IgniteConfiguration#setShutdownPolicy), on join we validate that
> statically configured policies on different server nodes are the same
> - It's possible to override statically configured value by adding
> distributed metastorage value, which can be done by
> calling ignite.cluster().setShutdownPolicy(plc) or control.sh method
> - Dynamic property is persisted
>
> Generally, I don't mind if we have both dynamic and static configuration
> properties. Necessity to call ignite.cluster().setShutdownPolicy(plc); on
> every new cluster creation is a usability issue itself.
> What bothers me here are the possible conflicts between static and dynamic
> configuration. User may be surprised if he has shutdown policy X in
> IgniteConfiguration, but the cluster behaves according to policy Y (because
> several months ago another admin had called
> IgniteCluster#setShutdownPolicy).
> We can handle it by adding a separate enum field to the shutdown policy:
>
>> public enum ShutdownPolicy {
>>   /* Default value of dynamic shutdown policy property. If it's set, the
>> shutdown policy is resolved according to value of static {@link
>> IgniteConfiguration#shutdownPolicy} configuration parameter. */
>>   USE_STATIC_CONFIGURATION,
>>
>>   /* Node leaves the cluster even if it's the last owner of some
>> partitions. Only partitions of caches with backups > 0 are taken into
>> account. */
>>   IMMEDIATE,
>>
>>   /* Shutdown is blocked until node is safe to leave without the data
>> loss. */
>>   GRACEFUL
>> }
>>
> This way:
> 1) User may easily understand whether the static parameter is overridden
> by dynamic. If ignite.cluster().getShutdownPolicy() return anything except
> USE_STATIC_CONFIGURATION, behavior is overridden.
> 2) User may clear previous overriding by calling
> ignite.cluster().setShutdownPolicy(USE_STATIC_CONFIGURATION). After that,
> behavior will be resolved based in IgniteConfiguration#shutdownPolicy again.
> If we agree on this mechanism, I propose to use IMMEDIATE name instead of
> DEFAULT for non-safe policy in order to don't confuse user.
> Meanwhile, static configuration will accept the same enum, but
> USE_STATIC_CONFIGURATION will be restricted:
>
>> public class IgniteConfiguration {
>>   public static final ShutdownPolicy DFLT_STATIC_SHUTDOWN_POLICY =
>> IMMEDIATE;
>>   private ShutdownPolicy shutdownPolicy = DFLT_STATIC_SHUTDOWN_POLICY;
>>   ...
>>   public void setShutdownPolicy(ShutdownPolicy shutdownPlc) {
>>     if (shutdownPlc ==  USE_STATIC_CONFIGURATION)
>>       throw new IllegalArgumentException("USE_STATIC_CONFIGURATION can
>> only be passed as dynamic property value via
>> ignite.cluster().setShutdownPolicy");
>>     ...
>>   }
>> ...
>> }
>>
>
> What do you think?
>
> On Tue, Jun 9, 2020 at 11:46 AM Alexei Scherbakov <
> alexey.scherbak...@gmail.com> wrote:
>
>> Ivan Rakov,
>>
>> Your proposal overall looks good to me. My comments:
>>
>> 1. I would avoid adding such a method, because it will be impossible to
>> change it in the future if some more shutdown policies will be introduced
>> later.
>> Also shutdown policy must be always consistent on the grid or
>> unintentional
>> data loss is possible if two nodes are stopping simultaneously with
>> different policies.
>>
>> This behavior can be achieved by changing policy globally when stopping a
>> node:
>> ignite.cluster().setShutdownPolicy(DEFAULT);
>> ignore.stop();
>>
>> 2. defaultShutdownPolicy with DEFAULT value is a mess. WAIT_FOR_BACKUPS is
>> not very clear either.
>> Let's use shutdownPolicy=DEFAULT|GRACEFUL, as was proposed by me earlier.
>>
>> 3. OK
>>
>> 4. OK
>>
>> 5. Let's keep a static property for simplifying setting of initial
>> behavior.
>> In most cases the policy will never be changed during grid's lifetime.
>> No need for an explicit call to API on grid start.
>> A joining node should check a local configuration value to match the grid.
>> If a dynamic value is already present in a metastore, it should override
>> static value with a warning.
>>
>>
>>
>>
>> пн, 8 июн. 2020 г. в 19:06, Ivan Rakov <ivan.glu...@gmail.com>:
>>
>> > Vlad, thanks for starting this discussion.
>> >
>> > I'll try to clarify the motivation for this change as I see it.
>> > In general, Ignite clusters are vulnerable to the data loss. Of course,
>> we
>> > have configurable PartitionLossPolicy, which allows to handle data loss
>> > safely and mitigate its consequences. But being able to avoid critical
>> > situations is always better than being able to recover from it.
>> >
>> > The most common issue from my perspective is absence of a way to perform
>> > rolling cluster restart safely. Scenario:
>> > 1. Backup count is 1
>> > 2. Admin wants to perform rolling restart in order to deploy new
>> version of
>> > business code that uses Ignite in embedded mode
>> > 3. Admin shuts down first node, replaces needed binaries and returns the
>> > node back to the topology
>> > 4. Node joins the cluster successfully
>> > 5. Admin shuts down second node
>> > 6. Data loss happens: the second node was the only owner of a certain
>> > partition, which was being rebalanced from the second node to the first
>> >
>> > We can prevent such situations by introducing "safe shutdown by default"
>> > mode, which blocks stopping node while it remains the only owner for at
>> > least one partition. It should be applied to "common" ways of stopping
>> > nodes - Ignite.close() and kill <pid>.
>> > I think, option to be enabled or disabled in runtime should be a
>> > requirement for this behavior. Safe shutdown mode has weird
>> side-effects.
>> > For example, admin won't be able to stop the whole cluster: stop of last
>> > node will be blocked, because the last node is the only present owner of
>> > all its partitions. Sure, kill -9 will resolve it, but it's still a
>> > usability issue.
>> >
>> > With the described dynamic property scenario will be changed as follows:
>> > 1. Admin enables "safe shutdown" mode
>> > 2. Admin shuts down first node, replaces needed binaries and returns the
>> > node back to the topology
>> > 3. Admin shuts down second node (with either ignite.close() or kill
>> <pid>),
>> > shutdown is blocked until the first node returns to the topology and
>> > completes the rebalancing process
>> > 4. Admin proceeds the rolling restart procedure
>> > 5. Admin disables "safe shutdown" mode
>> >
>> > This logic will also simplify the rolling restart scenario in K8S. Pod
>> with
>> > Ignite node won't be terminated until its termination will cause data
>> loss.
>> >
>> > Aside from waiting for backups, Ignition interface provide lots of
>> options
>> > to perform various node stop:
>> > - Whether or not to cancel pending compute jobs
>> > - Whether or not to perform instant halt() instead of any graceful stop
>> > logic
>> > - Whether or not to wait for some timeout before halt()
>> > - Whether or not the stopped grid should be restarted
>> > All these "stop" methods provide very custom logic. I don't see a need
>> to
>> > make them part of dynamic cluster-wide configuration. They still can be
>> > invoked directly via Java API. Later we can extract some of them to
>> dynamic
>> > cluster-wide parameters of default stop if it will become necessary.
>> That's
>> > why I think we should create an enum for default shutdown policy, but
>> only
>> > with two options so far (we can add more later): DEFAULT and
>> > WAIT_FOR_BACKUPS.
>> > Regarding the "NORMAL" option that you propose (where the node is not
>> > stopped until the rebalance is finished): I don't think that we should
>> add
>> > it. It doesn't ensure any strict guarantees: the data still can be lost
>> > with it.
>> >
>> > To sum it up, I propose:
>> > 1. Add a new method to Ignition interface to make it possible to stop
>> with
>> > "wait for backups" logic directly via Java API, like
>> Ignition.stop(boolean
>> > cancel, boolean waitForBackups)
>> > 2. Introduce "defaultShutdownPolicy" as a dynamic cluster configuration,
>> > two values are available so far: DEFAULT and WAIT_FOR_BACKUPS
>> > 3. This property is stored in the distributed metastorage (thus
>> persisted),
>> > can be changed via Java API and ./control.sh
>> > 4. Behavior configured with this property will be applied only on common
>> > ways of stopping the node - Ignite.close() and kill <pid>.
>> > 5. *Don't* add new options to the static IgniteConfiguration to avoid
>> > conflicts between dynamic and static configuration
>> >
>> > --
>> > Best Regards,
>> > Ivan Rakov
>> >
>> > On Mon, Jun 8, 2020 at 6:44 PM V.Pyatkov <vldpyat...@gmail.com> wrote:
>> >
>> > > Hi
>> > >
>> > > We need to have ability to calling shutdown with various guaranties.
>> > > For example:
>> > > Need to reboot a node, but after that node should be available for
>> > > historical rebalance (all partitions in MOVING state should have gone
>> to
>> > > OWNING).
>> > >
>> > > Implemented a circled reboot of cluster, but all data should be
>> available
>> > > on
>> > > that time (at least one copy of partition should be available in
>> > cluster).
>> > >
>> > > Need to wait not only data available, but all jobs (before this
>> behavior
>> > > available through a stop(false) method invocation).
>> > >
>> > > All these reason required various behavior before shutting down node.
>> > > I propose slightly modify public API and add here method which shown
>> on
>> > > shutdown behavior directly:
>> > > Ignite.close(Shutdown)
>> > >
>> > > /public enum Shutdownn {
>> > >     /**
>> > >      * Stop immediately as soon components are ready.
>> > >      */
>> > >     IMMEDIATE,
>> > >     /**
>> > >      * Stop node when all partitions completed moving from/to this
>> node
>> > to
>> > > another.
>> > >      */
>> > >     NORMAL,
>> > >     /**
>> > >      * Node will stop if and only if it does not store any unique
>> > > partitions, that does not have copies on cluster.
>> > >      */
>> > >     GRACEFUL,
>> > >     /**
>> > >      * Node stops graceful and wait all jobs before shutdown.
>> > >      */
>> > >     ALL
>> > > }/
>> > >
>> > > Method close without parameter Ignite.close() will get shutdown
>> behavior
>> > > configured for cluster wide. It will be implemented through
>> distributed
>> > > meta
>> > > storage and additional utilities for configuration.
>> > > Also, will be added a method to configure shutdown on start, this is
>> look
>> > > as
>> > > IgniteConfiguration.setShutdown(Shutdown).
>> > > If shutting down did not configure all be worked as before according
>> to
>> > > IMMEDIATE behavior.
>> > > All other close method will be marked as deprecated.
>> > >
>> > > I will be waiting for your opinions.
>> > >
>> > >
>> > >
>> > > --
>> > > Sent from: http://apache-ignite-developers.2346864.n4.nabble.com/
>> > >
>> >
>>
>>
>> --
>>
>> Best regards,
>> Alexei Scherbakov
>>
>

Reply via email to