Merge "Bug 6745 Remove thread renaming and unnecessary logging" into stable/boron
[openflowplugin.git] / openflowplugin-impl / src / main / java / org / opendaylight / openflowplugin / impl / device / DeviceManagerImpl.java
index 455278fa1e05de89abc5cd53508c08f0f023cb72..f3b14296e7495a33d5d977e6d42ffc2f4295deb2 100644 (file)
@@ -8,19 +8,17 @@
 package org.opendaylight.openflowplugin.impl.device;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Verify;
 import com.google.common.collect.Iterators;
-import com.google.common.util.concurrent.AsyncFunction;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.util.HashedWheelTimer;
 import io.netty.util.TimerTask;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.Iterator;
-import java.util.Set;
+import java.util.Objects;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutionException;
@@ -29,34 +27,33 @@ import java.util.concurrent.TimeUnit;
 import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
 import org.opendaylight.controller.md.sal.binding.api.DataBroker;
+import org.opendaylight.controller.md.sal.binding.api.NotificationPublishService;
 import org.opendaylight.controller.md.sal.binding.api.WriteTransaction;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
+import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider;
 import org.opendaylight.openflowjava.protocol.api.connection.ConnectionAdapter;
 import org.opendaylight.openflowjava.protocol.api.connection.OutboundQueueHandlerRegistration;
 import org.opendaylight.openflowplugin.api.openflow.OFPContext;
 import org.opendaylight.openflowplugin.api.openflow.connection.ConnectionContext;
+import org.opendaylight.openflowplugin.api.openflow.connection.ConnectionStatus;
 import org.opendaylight.openflowplugin.api.openflow.connection.OutboundQueueProvider;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceContext;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceInfo;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceManager;
-import org.opendaylight.openflowplugin.api.openflow.device.DeviceState;
-import org.opendaylight.openflowplugin.api.openflow.device.DeviceSynchronizeListener;
-import org.opendaylight.openflowplugin.api.openflow.device.DeviceValidListener;
 import org.opendaylight.openflowplugin.api.openflow.device.TranslatorLibrary;
 import org.opendaylight.openflowplugin.api.openflow.device.handlers.DeviceInitializationPhaseHandler;
 import org.opendaylight.openflowplugin.api.openflow.device.handlers.DeviceTerminationPhaseHandler;
-import org.opendaylight.openflowplugin.api.openflow.lifecycle.LifecycleConductor;
-import org.opendaylight.openflowplugin.api.openflow.statistics.StatisticsContext;
+import org.opendaylight.openflowplugin.api.openflow.lifecycle.LifecycleService;
+import org.opendaylight.openflowplugin.api.openflow.statistics.ofpspecific.MessageSpy;
 import org.opendaylight.openflowplugin.extension.api.ExtensionConverterProviderKeeper;
 import org.opendaylight.openflowplugin.extension.api.core.extension.ExtensionConverterProvider;
 import org.opendaylight.openflowplugin.impl.connection.OutboundQueueProviderImpl;
 import org.opendaylight.openflowplugin.impl.device.listener.OpenflowProtocolListenerFullImpl;
-import org.opendaylight.openflowplugin.impl.util.DeviceInitializationUtils;
+import org.opendaylight.openflowplugin.impl.lifecycle.LifecycleServiceImpl;
 import org.opendaylight.openflowplugin.openflow.md.core.sal.convertor.ConvertorExecutor;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodesBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.Node;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.role.service.rev150727.OfpRole;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -71,7 +68,7 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
     private final long globalNotificationQuota;
     private final boolean switchFeaturesMandatory;
     private boolean isNotificationFlowRemovedOff;
-
+    private boolean skipTableFeatures;
     private static final int SPY_RATE = 10;
 
     private final DataBroker dataBroker;
@@ -81,29 +78,35 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
     private DeviceTerminationPhaseHandler deviceTerminPhaseHandler;
 
     private final ConcurrentMap<DeviceInfo, DeviceContext> deviceContexts = new ConcurrentHashMap<>();
+    private final ConcurrentMap<DeviceInfo, LifecycleService> lifecycleServices = new ConcurrentHashMap<>();
 
     private final long barrierIntervalNanos;
     private final int barrierCountLimit;
+
     private ExtensionConverterProvider extensionConverterProvider;
     private ScheduledThreadPoolExecutor spyPool;
-    private Set<DeviceSynchronizeListener> deviceSynchronizedListeners;
-    private Set<DeviceValidListener> deviceValidListeners;
-
-    private final LifecycleConductor conductor;
+    private final ClusterSingletonServiceProvider singletonServiceProvider;
+    private final NotificationPublishService notificationPublishService;
+    private final MessageSpy messageSpy;
+    private final HashedWheelTimer hashedWheelTimer;
 
     public DeviceManagerImpl(@Nonnull final DataBroker dataBroker,
-                             final long globalNotificationQuota, final boolean switchFeaturesMandatory,
-                             final long barrierInterval, final int barrierCountLimit,
-                             final LifecycleConductor lifecycleConductor, boolean isNotificationFlowRemovedOff,
-                             final ConvertorExecutor convertorExecutor) {
-        this.switchFeaturesMandatory = switchFeaturesMandatory;
-        this.globalNotificationQuota = globalNotificationQuota;
-        this.isNotificationFlowRemovedOff = isNotificationFlowRemovedOff;
-        this.dataBroker = Preconditions.checkNotNull(dataBroker);
-        this.convertorExecutor = convertorExecutor;
+                             final long globalNotificationQuota,
+                             final boolean switchFeaturesMandatory,
+                             final long barrierInterval,
+                             final int barrierCountLimit,
+                             final MessageSpy messageSpy,
+                             final boolean isNotificationFlowRemovedOff,
+                             final ClusterSingletonServiceProvider singletonServiceProvider,
+                             final NotificationPublishService notificationPublishService,
+                             final HashedWheelTimer hashedWheelTimer,
+                             final ConvertorExecutor convertorExecutor,
+                             final boolean skipTableFeatures) {
+
+        this.dataBroker = dataBroker;
+
         /* merge empty nodes to oper DS to predict any problems with missing parent for Node */
         final WriteTransaction tx = dataBroker.newWriteOnlyTransaction();
-
         final NodesBuilder nodesBuilder = new NodesBuilder();
         nodesBuilder.setNode(Collections.<Node>emptyList());
         tx.merge(LogicalDatastoreType.OPERATIONAL, InstanceIdentifier.create(Nodes.class), nodesBuilder.build());
@@ -114,13 +117,18 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
             throw new IllegalStateException(e);
         }
 
+        this.switchFeaturesMandatory = switchFeaturesMandatory;
+        this.globalNotificationQuota = globalNotificationQuota;
+        this.isNotificationFlowRemovedOff = isNotificationFlowRemovedOff;
+        this.skipTableFeatures = skipTableFeatures;
+        this.convertorExecutor = convertorExecutor;
+        this.hashedWheelTimer = hashedWheelTimer;
         this.barrierIntervalNanos = TimeUnit.MILLISECONDS.toNanos(barrierInterval);
         this.barrierCountLimit = barrierCountLimit;
-
-        this.conductor = lifecycleConductor;
-        spyPool = new ScheduledThreadPoolExecutor(1);
-        this.deviceSynchronizedListeners = new HashSet<>();
-        this.deviceValidListeners = new HashSet<>();
+        this.spyPool = new ScheduledThreadPoolExecutor(1);
+        this.singletonServiceProvider = singletonServiceProvider;
+        this.notificationPublishService = notificationPublishService;
+        this.messageSpy = messageSpy;
     }
 
 
@@ -130,26 +138,34 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
     }
 
     @Override
-    public void onDeviceContextLevelUp(@CheckForNull DeviceInfo deviceInfo) throws Exception {
+    public void onDeviceContextLevelUp(@CheckForNull DeviceInfo deviceInfo, final LifecycleService lifecycleService) throws Exception {
         // final phase - we have to add new Device to MD-SAL DataStore
         LOG.debug("Final phase of DeviceContextLevelUp for Node: {} ", deviceInfo.getNodeId());
         DeviceContext deviceContext = Preconditions.checkNotNull(deviceContexts.get(deviceInfo));
         deviceContext.onPublished();
+        lifecycleService.registerService(this.singletonServiceProvider);
     }
 
     @Override
-    public boolean deviceConnected(@CheckForNull final ConnectionContext connectionContext) throws Exception {
+    public ConnectionStatus deviceConnected(@CheckForNull final ConnectionContext connectionContext) throws Exception {
         Preconditions.checkArgument(connectionContext != null);
 
         DeviceInfo deviceInfo = connectionContext.getDeviceInfo();
-        /**
+        /*
          * This part prevent destroy another device context. Throwing here an exception result to propagate close connection
          * in {@link org.opendaylight.openflowplugin.impl.connection.org.opendaylight.openflowplugin.impl.connection.HandshakeContextImpl}
          * If context already exist we are in state closing process (connection flapping) and we should not propagate connection close
          */
          if (deviceContexts.containsKey(deviceInfo)) {
-            LOG.warn("Rejecting connection from node which is already connected and there exist deviceContext for it: {}", connectionContext.getNodeId());
-             return false;
+             DeviceContext deviceContext = deviceContexts.get(deviceInfo);
+             LOG.warn("Node {} already connected disconnecting device. Rejecting connection", deviceInfo.getLOGValue());
+             if (!deviceContext.getState().equals(OFPContext.CONTEXT_STATE.TERMINATION)) {
+                 LOG.warn("Node {} context state not in TERMINATION state.",
+                         connectionContext.getDeviceInfo().getLOGValue());
+                 return ConnectionStatus.ALREADY_CONNECTED;
+             } else {
+                 return ConnectionStatus.CLOSING;
+             }
          }
 
         LOG.info("ConnectionEvent: Device connected to controller, Device:{}, NodeId:{}",
@@ -170,35 +186,36 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
                 connectionAdapter.registerOutboundQueueHandler(outboundQueueProvider, barrierCountLimit, barrierIntervalNanos);
         connectionContext.setOutboundQueueHandleRegistration(outboundQueueHandlerRegistration);
 
-        final DeviceState deviceState = new DeviceStateImpl(deviceInfo);
-        this.addDeviceSynchronizeListener(deviceState);
-        this.addDeviceValidListener(deviceState);
-
-        final DeviceContext deviceContext = new DeviceContextImpl(connectionContext,
-                deviceState,
+        final DeviceContext deviceContext = new DeviceContextImpl(
+                connectionContext,
                 dataBroker,
-                conductor,
-                outboundQueueProvider,
+                messageSpy,
                 translatorLibrary,
-                this);
+                this,
+                convertorExecutor,
+                skipTableFeatures);
 
-        Verify.verify(deviceContexts.putIfAbsent(deviceInfo, deviceContext) == null, "DeviceCtx still not closed.");
+        deviceContexts.put(deviceInfo, deviceContext);
+
+        final LifecycleService lifecycleService = new LifecycleServiceImpl();
+        lifecycleService.setDeviceContext(deviceContext);
+        deviceContext.putLifecycleServiceIntoTxChainManager(lifecycleService);
+
+        lifecycleServices.put(deviceInfo, lifecycleService);
+
+        deviceContext.setSwitchFeaturesMandatory(switchFeaturesMandatory);
 
         ((ExtensionConverterProviderKeeper) deviceContext).setExtensionConverterProvider(extensionConverterProvider);
-        deviceContext.setNotificationPublishService(conductor.getNotificationPublishService());
+        deviceContext.setNotificationPublishService(notificationPublishService);
 
         updatePacketInRateLimiters();
 
         final OpenflowProtocolListenerFullImpl messageListener = new OpenflowProtocolListenerFullImpl(
                 connectionAdapter, deviceContext);
-        connectionAdapter.setMessageListener(messageListener);
-        notifyDeviceValidListeners(deviceInfo, true);
 
-        deviceInitPhaseHandler.onDeviceContextLevelUp(connectionContext.getDeviceInfo());
-
-        notifyDeviceSynchronizeListeners(deviceInfo, true);
-
-        return true;
+        connectionAdapter.setMessageListener(messageListener);
+        deviceInitPhaseHandler.onDeviceContextLevelUp(connectionContext.getDeviceInfo(), lifecycleService);
+        return ConnectionStatus.MAY_CONTINUE;
     }
 
     private void updatePacketInRateLimiters() {
@@ -209,7 +226,9 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
                 if (freshNotificationLimit < 100) {
                     freshNotificationLimit = 100;
                 }
-                LOG.debug("fresh notification limit = {}", freshNotificationLimit);
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("fresh notification limit = {}", freshNotificationLimit);
+                }
                 for (final DeviceContext deviceContext : deviceContexts.values()) {
                     deviceContext.updatePacketInRateLimit(freshNotificationLimit);
                 }
@@ -232,27 +251,43 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
         for (final Iterator<DeviceContext> iterator = Iterators.consumingIterator(deviceContexts.values().iterator());
                 iterator.hasNext();) {
             final DeviceContext deviceCtx = iterator.next();
-            notifyDeviceValidListeners(deviceCtx.getDeviceInfo(), false);
             deviceCtx.shutdownConnection();
             deviceCtx.shuttingDownDataStoreTransactions();
         }
 
-        if (spyPool != null) {
-            spyPool.shutdownNow();
-            spyPool = null;
-        }
+        Optional.ofNullable(spyPool).ifPresent(ScheduledThreadPoolExecutor::shutdownNow);
+        spyPool = null;
+
     }
 
     @Override
     public void onDeviceContextLevelDown(final DeviceInfo deviceInfo) {
-        LOG.debug("onDeviceContextClosed for Node {}", deviceInfo.getNodeId());
-        deviceContexts.remove(deviceInfo);
+
+        LifecycleService lifecycleService = lifecycleServices.remove(deviceInfo);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Lifecycle service removed for node {}", deviceInfo.getLOGValue());
+        }
+
         updatePacketInRateLimiters();
+        if (Objects.nonNull(lifecycleService)) {
+            try {
+                lifecycleService.close();
+                LOG.debug("Lifecycle service successfully closed for node {}", deviceInfo.getLOGValue());
+            } catch (Exception e) {
+                LOG.warn("Closing lifecycle service for node {} was unsuccessful ", deviceInfo.getLOGValue(), e);
+            }
+        }
+
+        deviceContexts.remove(deviceInfo);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Device context removed for node {}", deviceInfo.getLOGValue());
+        }
+
     }
 
     @Override
     public void initialize() {
-        spyPool.scheduleAtFixedRate(conductor.getMessageIntelligenceAgency(), SPY_RATE, SPY_RATE, TimeUnit.SECONDS);
+        spyPool.scheduleAtFixedRate(messageSpy, SPY_RATE, SPY_RATE, TimeUnit.SECONDS);
     }
 
     @Override
@@ -277,40 +312,48 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
         final DeviceContext deviceCtx = this.deviceContexts.get(deviceInfo);
 
         if (null == deviceCtx) {
-            LOG.info("DeviceContext for Node {} was not found. Connection is terminated without OFP context suite.", deviceInfo.getNodeId());
+            LOG.info("DeviceContext for Node {} was not found. Connection is terminated without OFP context suite.", deviceInfo.getLOGValue());
+            return;
+        }
+
+        if (deviceCtx.getState().equals(OFPContext.CONTEXT_STATE.TERMINATION)) {
+            LOG.info("Device context for node {} is already is termination state, waiting for close all context", deviceInfo.getLOGValue());
             return;
         }
 
+        deviceCtx.setState(OFPContext.CONTEXT_STATE.TERMINATION);
+
         if (!connectionContext.equals(deviceCtx.getPrimaryConnectionContext())) {
+            LOG.debug("Node {} disconnected, but not primary connection.", connectionContext.getDeviceInfo().getLOGValue());
             /* Connection is not PrimaryConnection so try to remove from Auxiliary Connections */
             deviceCtx.removeAuxiliaryConnectionContext(connectionContext);
-        } else {
-            notifyDeviceValidListeners(deviceInfo, false);
+        }
+        //TODO: Auxiliary connections supported ?
             /* Device is disconnected and so we need to close TxManager */
-            final ListenableFuture<Void> future = deviceCtx.shuttingDownDataStoreTransactions();
-            Futures.addCallback(future, new FutureCallback<Void>() {
+        final ListenableFuture<Void> future = deviceCtx.shuttingDownDataStoreTransactions();
+        Futures.addCallback(future, new FutureCallback<Void>() {
 
-                @Override
-                public void onSuccess(final Void result) {
-                    LOG.debug("TxChainManager for device {} is closed successful.", deviceInfo.getNodeId());
-                    deviceTerminPhaseHandler.onDeviceContextLevelDown(deviceInfo);
-                }
+            @Override
+            public void onSuccess(final Void result) {
+                LOG.debug("TxChainManager for device {} is closed successful.", deviceInfo.getLOGValue());
+                deviceTerminPhaseHandler.onDeviceContextLevelDown(deviceInfo);
+            }
 
-                @Override
-                public void onFailure(final Throwable t) {
-                    LOG.warn("TxChainManager for device {} failed by closing.", deviceInfo.getNodeId(), t);
-                    deviceTerminPhaseHandler.onDeviceContextLevelDown(deviceInfo);
-                }
-            });
-            /* Add timer for Close TxManager because it could fain ind cluster without notification */
-            final TimerTask timerTask = timeout -> {
-                if (!future.isDone()) {
-                    LOG.info("Shutting down TxChain for node {} not completed during 10 sec. Continue anyway.", deviceInfo.getNodeId());
-                    future.cancel(false);
-                }
-            };
-            conductor.newTimeout(timerTask, 10, TimeUnit.SECONDS);
-        }
+            @Override
+            public void onFailure(final Throwable t) {
+                LOG.warn("TxChainManager for device {} failed by closing.", deviceInfo.getLOGValue());
+                LOG.trace("TxChainManager failed by closing. ", t);
+                deviceTerminPhaseHandler.onDeviceContextLevelDown(deviceInfo);
+            }
+        });
+        /* Add timer for Close TxManager because it could fain ind cluster without notification */
+        final TimerTask timerTask = timeout -> {
+            if (!future.isDone()) {
+                LOG.warn("Shutting down TxChain for node {} not completed during 10 sec. Continue anyway.", deviceInfo.getLOGValue());
+                future.cancel(false);
+            }
+        };
+        hashedWheelTimer.newTimeout(timerTask, 10, TimeUnit.SECONDS);
     }
 
     @VisibleForTesting
@@ -318,45 +361,6 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
         deviceContexts.put(deviceInfo, deviceContext);
     }
 
-    @Override
-    public <T extends OFPContext> T gainContext(final DeviceInfo deviceInfo) {
-        return (T) deviceContexts.get(deviceInfo);
-    }
-
-    @Override
-    public ListenableFuture<Void> onClusterRoleChange(final DeviceInfo deviceInfo, final OfpRole role) {
-        DeviceContext deviceContext = deviceContexts.get(deviceInfo);
-        LOG.trace("onClusterRoleChange {} for node:", role, deviceInfo.getNodeId());
-        if (OfpRole.BECOMEMASTER.equals(role)) {
-            return onDeviceTakeClusterLeadership(deviceInfo);
-        }
-        return ((DeviceContextImpl)deviceContext).getTransactionChainManager().deactivateTransactionManager();
-    }
-
-    @Override
-    public void addDeviceSynchronizeListener(final DeviceSynchronizeListener deviceSynchronizeListener) {
-        this.deviceSynchronizedListeners.add(deviceSynchronizeListener);
-    }
-
-    @Override
-    public void notifyDeviceSynchronizeListeners(final DeviceInfo deviceInfo, final boolean deviceSynchronized) {
-        for (DeviceSynchronizeListener listener : deviceSynchronizedListeners) {
-            listener.deviceIsSynchronized(deviceInfo, deviceSynchronized);
-        }
-    }
-
-    @Override
-    public void addDeviceValidListener(final DeviceValidListener deviceValidListener) {
-        this.deviceValidListeners.add(deviceValidListener);
-    }
-
-    @Override
-    public void notifyDeviceValidListeners(final DeviceInfo deviceInfo, final boolean deviceValid) {
-        for (DeviceValidListener listener : deviceValidListeners) {
-            listener.deviceIsValid(deviceInfo, deviceValid);
-        }
-    }
-
     @Override
     public void setIsNotificationFlowRemovedOff(boolean isNotificationFlowRemovedOff) {
         this.isNotificationFlowRemovedOff = isNotificationFlowRemovedOff;
@@ -367,62 +371,10 @@ public class DeviceManagerImpl implements DeviceManager, ExtensionConverterProvi
         return this.isNotificationFlowRemovedOff;
     }
 
-    private ListenableFuture<Void> onDeviceTakeClusterLeadership(final DeviceInfo deviceInfo) {
-        LOG.trace("onDeviceTakeClusterLeadership for node: {}", deviceInfo.getNodeId());
-        /* validation */
-        StatisticsContext statisticsContext = conductor.getStatisticsContext(deviceInfo);
-        if (statisticsContext == null) {
-            final String errMsg = String.format("DeviceCtx %s is up but we are missing StatisticsContext", deviceInfo.getDatapathId());
-            LOG.warn(errMsg);
-            return Futures.immediateFailedFuture(new IllegalStateException(errMsg));
-        }
-        DeviceContext deviceContext = deviceContexts.get(deviceInfo);
-        /* Prepare init info collecting */
-        notifyDeviceSynchronizeListeners(deviceInfo, false);
-        ((DeviceContextImpl)deviceContext).getTransactionChainManager().activateTransactionManager();
-        ((DeviceContextImpl)deviceContext).getTransactionChainManager().initialSubmitWriteTransaction();
-        /* Init Collecting NodeInfo */
-        final ListenableFuture<Void> initCollectingDeviceInfo = DeviceInitializationUtils.initializeNodeInformation(
-                deviceContext, switchFeaturesMandatory, convertorExecutor);
-        /* Init Collecting StatInfo */
-        final ListenableFuture<Boolean> statPollFuture = Futures.transform(initCollectingDeviceInfo,
-                new AsyncFunction<Void, Boolean>() {
-
-                    @Override
-                    public ListenableFuture<Boolean> apply(@Nonnull final Void input) throws Exception {
-                        statisticsContext.statListForCollectingInitialization();
-                        return statisticsContext.initialGatherDynamicData();
-                    }
-                });
-
-        return Futures.transform(statPollFuture, getInitialDeviceInformation(deviceContext));
-    }
 
-    private Function<Boolean, Void> getInitialDeviceInformation(final DeviceContext deviceContext) {
-        return input -> {
-            if (ConnectionContext.CONNECTION_STATE.RIP.equals(
-                    conductor.gainConnectionStateSafely(deviceContext.getDeviceInfo())
-            )) {
-                final String errMsg =
-                        String.format("We lost connection for Device %s, context has to be closed.",
-                        deviceContext.getDeviceInfo().getNodeId());
-                LOG.warn(errMsg);
-                throw new IllegalStateException(errMsg);
-            }
-
-            if (input == null || !input) {
-                final String errMsg =
-                        String.format("Get Initial Device %s information fails",
-                        deviceContext.getDeviceInfo().getNodeId());
-                LOG.warn(errMsg);
-                throw new IllegalStateException(errMsg);
-            }
-            LOG.debug("Get Initial Device {} information is successful",
-                    deviceContext.getDeviceInfo().getNodeId());
-            notifyDeviceSynchronizeListeners(deviceContext.getDeviceInfo(), true);
-            deviceContext.getDeviceState().setStatisticsPollingEnabledProp(true);
-            return null;
-        };
+    @Override
+    public void setSkipTableFeatures(boolean skipTableFeaturesValue) {
+        skipTableFeatures = skipTableFeaturesValue;
     }
 
 }