Partial rebind - basically working, but for mgmt confusion makes RebindIteration abstract, with InitialFullRI having the old (full) behaviour, and new ActivePartialRebindIteration supporting a subset of nodes; minor fixes elsewhere to prevent problems
Project: http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/commit/0fb0c633 Tree: http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/tree/0fb0c633 Diff: http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/diff/0fb0c633 Branch: refs/heads/master Commit: 0fb0c633d269f48cf474c036b8c14bfaba03b5c5 Parents: 0d078c9 Author: Alex Heneveld <alex.henev...@cloudsoftcorp.com> Authored: Thu Jan 29 11:26:13 2015 +0000 Committer: Alex Heneveld <alex.henev...@cloudsoftcorp.com> Committed: Fri Feb 6 22:12:12 2015 +0000 ---------------------------------------------------------------------- .../rebind/ActivePartialRebindIteration.java | 161 ++++++++++++++ .../rebind/InitialFullRebindIteration.java | 130 ++++++++++++ .../rebind/RebindContextLookupContext.java | 24 +++ .../brooklyn/entity/rebind/RebindIteration.java | 208 +++++++------------ .../entity/rebind/RebindManagerImpl.java | 58 +++++- .../BrooklynMementoPersisterToObjectStore.java | 4 + .../internal/AbstractManagementContext.java | 6 +- .../rebind/RebindOnlySomeEntitiesTest.java | 44 ++++ .../entity/database/mysql/MySqlNodeImpl.java | 2 +- .../brooklyn/rest/resources/ScriptResource.java | 1 + .../rest/util/BrooklynRestResourceUtils.java | 2 +- 11 files changed, 500 insertions(+), 140 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java b/core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java new file mode 100644 index 0000000..fdcc810 --- /dev/null +++ b/core/src/main/java/brooklyn/entity/rebind/ActivePartialRebindIteration.java @@ -0,0 +1,161 @@ +/* + * 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 brooklyn.entity.rebind; + +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import brooklyn.basic.BrooklynObject; +import brooklyn.basic.BrooklynObjectInternal; +import brooklyn.entity.Entity; +import brooklyn.entity.basic.EntityInternal; +import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore; +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics; +import brooklyn.entity.rebind.transformer.CompoundTransformer; +import brooklyn.management.ha.ManagementNodeState; +import brooklyn.mementos.BrooklynMementoPersister; +import brooklyn.mementos.BrooklynMementoRawData; +import brooklyn.mementos.BrooklynMementoRawData.Builder; +import brooklyn.mementos.Memento; +import brooklyn.util.collections.MutableList; +import brooklyn.util.collections.MutableSet; + +import com.google.common.base.Preconditions; + +/** + * Pauses a set of existing entities, writes their state, applies a transformation, then reads them back. + */ +public class ActivePartialRebindIteration extends RebindIteration { + + private static final Logger LOG = LoggerFactory.getLogger(ActivePartialRebindIteration.class); + + protected Iterator<BrooklynObject> objectsToRebindInitial; + protected Collection<BrooklynObject> objectsToRebindFinal; + protected List<CompoundTransformer> transformers = MutableList.of(); + + public ActivePartialRebindIteration(RebindManagerImpl rebindManager, + ManagementNodeState mode, + ClassLoader classLoader, RebindExceptionHandler exceptionHandler, + Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess + ) { + super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess); + } + + @Override + protected boolean isRebindingActiveAgain() { + return true; + } + + public void setObjectIterator(Iterator<BrooklynObject> objectsToRebind) { + this.objectsToRebindInitial = objectsToRebind; + } + + public void applyTransformer(CompoundTransformer transformer) { + transformers.add(transformer); + } + + @Override + protected void doRun() throws Exception { + checkEnteringPhase(1); + Preconditions.checkState(readOnlyRebindCount.get()==Integer.MIN_VALUE, "Partial rebind in read-only mode not supported"); + Preconditions.checkNotNull(objectsToRebindInitial, "Objects to rebind must be set"); + + LOG.debug("Partial rebind Rebinding ("+mode+") from "+rebindManager.getPersister().getBackingStoreDescription()+"..."); + + Builder mementoRawBuilder = BrooklynMementoRawData.builder(); + + /* + * TODO detail... + * This unmanages and re-manages. Not sure if that's ideal. + * Probably we should try to pause it, or switch to a model + * where each entity can be managed by any node. + */ + + objectsToRebindFinal = MutableSet.of(); + while (objectsToRebindInitial.hasNext()) { + BrooklynObject bo = objectsToRebindInitial.next(); + objectsToRebindFinal.add(bo); + + if (bo instanceof Entity) { + // if it's an entity, add all adjuncts. (if doing some sort of pause, that's maybe not necessary...) + objectsToRebindFinal.addAll( ((EntityInternal)bo).getPolicies() ); + objectsToRebindFinal.addAll( ((EntityInternal)bo).getEnrichers() ); + objectsToRebindFinal.addAll( ((EntityInternal)bo).feeds().getFeeds() ); + } + } + + // TODO unmanage? pause? + // get serialization + for (BrooklynObject bo: objectsToRebindFinal) { + Memento m = ((BrooklynObjectInternal)bo).getRebindSupport().getMemento(); + BrooklynMementoPersister p = rebindManager.getPersister(); + String mr = ((BrooklynMementoPersisterToObjectStore)p).getMementoSerializer().toString(m); + mementoRawBuilder.put(BrooklynObjectType.of(bo), bo.getId(), mr); + } + + // then rebuild + mementoRawData = mementoRawBuilder.build(); + // TODO see comment in InitialFullRebindIteration + mementoManifest = persistenceStoreAccess.loadMementoManifest(mementoRawData, exceptionHandler); + + determineStateFromManifestFiles(); + + // skip this phase, as catalog is not being changed + // (and we don't want to unload things) +// rebuildCatalog(); + phase++; + + instantiateLocationsAndEntities(); + instantiateMementos(); + instantiateAdjuncts(instantiator); + reconstructEverything(); + associateAdjunctsWithEntities(); + manageTheObjects(); + finishingUp(); + } + + @Override + protected void determineStateFromManifestFiles() { + super.determineStateFromManifestFiles(); + overwritingMaster = true; + } + + @Override + protected Collection<String> getMementoRootEntities() { + // TODO must parents + return memento.getEntityIds(); + } + + @Override + protected void cleanupOldLocations(Set<String> oldLocations) { + // not applicable here + } + @Override + protected void cleanupOldEntities(Set<String> oldEntities) { + // not applicable here + } + +} http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/core/src/main/java/brooklyn/entity/rebind/InitialFullRebindIteration.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/brooklyn/entity/rebind/InitialFullRebindIteration.java b/core/src/main/java/brooklyn/entity/rebind/InitialFullRebindIteration.java new file mode 100644 index 0000000..9b08392 --- /dev/null +++ b/core/src/main/java/brooklyn/entity/rebind/InitialFullRebindIteration.java @@ -0,0 +1,130 @@ +/* + * 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 brooklyn.entity.rebind; + +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import brooklyn.config.BrooklynLogging; +import brooklyn.entity.Entity; +import brooklyn.entity.basic.EntityInternal; +import brooklyn.entity.rebind.persister.PersistenceActivityMetrics; +import brooklyn.management.ha.ManagementNodeState; +import brooklyn.management.internal.EntityManagerInternal; +import brooklyn.management.internal.LocationManagerInternal; +import brooklyn.management.internal.ManagementTransitionInfo.ManagementTransitionMode; +import brooklyn.mementos.BrooklynMementoPersister; +import brooklyn.util.text.Strings; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; + +/** + * Does an un-bind (if necessary) and re-bind for a subset of items. + */ +public class InitialFullRebindIteration extends RebindIteration { + + private static final Logger LOG = LoggerFactory.getLogger(InitialFullRebindIteration.class); + + public InitialFullRebindIteration(RebindManagerImpl rebindManager, + ManagementNodeState mode, + ClassLoader classLoader, RebindExceptionHandler exceptionHandler, + Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess + ) { + super(rebindManager, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess); + } + + @Override + protected boolean isRebindingActiveAgain() { + return false; + } + + protected void doRun() throws Exception { + LOG.debug("Rebinding ("+mode+ + (readOnlyRebindCount.get()>Integer.MIN_VALUE ? ", iteration "+readOnlyRebindCount : "")+ + ") from "+rebindManager.getPersister().getBackingStoreDescription()+"..."); + + loadManifestFiles(); + rebuildCatalog(); + instantiateLocationsAndEntities(); + instantiateMementos(); + instantiateAdjuncts(instantiator); + reconstructEverything(); + associateAdjunctsWithEntities(); + manageTheObjects(); + finishingUp(); + } + + protected void loadManifestFiles() throws Exception { + checkEnteringPhase(1); + Preconditions.checkState(mementoRawData==null && mementoManifest==null, "Memento data should not yet be set when calling this"); + + mementoRawData = persistenceStoreAccess.loadMementoRawData(exceptionHandler); + + // TODO building the manifests should be part of this class (or parent) + // it does not have anything to do with the persistence store! + mementoManifest = persistenceStoreAccess.loadMementoManifest(mementoRawData, exceptionHandler); + + determineStateFromManifestFiles(); + + if (mode!=ManagementNodeState.HOT_STANDBY && mode!=ManagementNodeState.HOT_BACKUP) { + if (!isEmpty) { + LOG.info("Rebinding from "+getPersister().getBackingStoreDescription()+" for "+Strings.toLowerCase(Strings.toString(mode))+" "+managementContext.getManagementNodeId()+"..."); + } else { + LOG.info("Rebind check: no existing state; will persist new items to "+getPersister().getBackingStoreDescription()); + } + + if (!managementContext.getEntityManager().getEntities().isEmpty() || !managementContext.getLocationManager().getLocations().isEmpty()) { + // this is discouraged if we were already master + Entity anEntity = Iterables.getFirst(managementContext.getEntityManager().getEntities(), null); + if (anEntity!=null && !((EntityInternal)anEntity).getManagementSupport().isReadOnly()) { + overwritingMaster = true; + LOG.warn("Rebind requested for "+mode+" node "+managementContext.getManagementNodeId()+" " + + "when it already has active state; discouraged, " + + "will likely overwrite: "+managementContext.getEntityManager().getEntities()+" and "+managementContext.getLocationManager().getLocations()+" and more"); + } + } + } + } + + @Override + protected void cleanupOldLocations(Set<String> oldLocations) { + LocationManagerInternal locationManager = (LocationManagerInternal)managementContext.getLocationManager(); + if (!oldLocations.isEmpty()) BrooklynLogging.log(LOG, overwritingMaster ? BrooklynLogging.LoggingLevel.WARN : BrooklynLogging.LoggingLevel.DEBUG, + "Destroying unused locations on rebind: "+oldLocations); + for (String oldLocationId: oldLocations) { + locationManager.unmanage(locationManager.getLocation(oldLocationId), ManagementTransitionMode.REBINDING_DESTROYED); + } + } + + @Override + protected void cleanupOldEntities(Set<String> oldEntities) { + EntityManagerInternal entityManager = (EntityManagerInternal)managementContext.getEntityManager(); + if (!oldEntities.isEmpty()) BrooklynLogging.log(LOG, overwritingMaster ? BrooklynLogging.LoggingLevel.WARN : BrooklynLogging.LoggingLevel.DEBUG, + "Destroying unused entities on rebind: "+oldEntities); + for (String oldEntityId: oldEntities) { + entityManager.unmanage(entityManager.getEntity(oldEntityId), ManagementTransitionMode.REBINDING_DESTROYED); + } + } + +} http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/core/src/main/java/brooklyn/entity/rebind/RebindContextLookupContext.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/brooklyn/entity/rebind/RebindContextLookupContext.java b/core/src/main/java/brooklyn/entity/rebind/RebindContextLookupContext.java index 6392a5f..05730d7 100644 --- a/core/src/main/java/brooklyn/entity/rebind/RebindContextLookupContext.java +++ b/core/src/main/java/brooklyn/entity/rebind/RebindContextLookupContext.java @@ -25,6 +25,7 @@ import org.slf4j.LoggerFactory; import brooklyn.basic.BrooklynObject; import brooklyn.catalog.CatalogItem; +import brooklyn.catalog.internal.CatalogUtils; import brooklyn.entity.Entity; import brooklyn.entity.Feed; import brooklyn.location.Location; @@ -43,14 +44,19 @@ public class RebindContextLookupContext implements LookupContext { protected final RebindContextImpl rebindContext; protected final RebindExceptionHandler exceptionHandler; + protected final boolean lookInManagementContext; public RebindContextLookupContext(RebindContextImpl rebindContext, RebindExceptionHandler exceptionHandler) { this(null, rebindContext, exceptionHandler); } public RebindContextLookupContext(ManagementContext managementContext, RebindContextImpl rebindContext, RebindExceptionHandler exceptionHandler) { + this(managementContext, rebindContext, exceptionHandler, false); + } + public RebindContextLookupContext(ManagementContext managementContext, RebindContextImpl rebindContext, RebindExceptionHandler exceptionHandler, boolean lookInManagementContext) { this.managementContext = managementContext; this.rebindContext = rebindContext; this.exceptionHandler = exceptionHandler; + this.lookInManagementContext = lookInManagementContext; } @Override public ManagementContext lookupManagementContext() { @@ -60,6 +66,9 @@ public class RebindContextLookupContext implements LookupContext { @Override public Entity lookupEntity(String id) { Entity result = rebindContext.getEntity(id); if (result == null) { + result = managementContext.lookup(id, Entity.class); + } + if (result == null) { result = exceptionHandler.onDanglingEntityRef(id); } return result; @@ -68,6 +77,9 @@ public class RebindContextLookupContext implements LookupContext { @Override public Location lookupLocation(String id) { Location result = rebindContext.getLocation(id); if (result == null) { + result = managementContext.lookup(id, Location.class); + } + if (result == null) { result = exceptionHandler.onDanglingLocationRef(id); } return result; @@ -76,6 +88,9 @@ public class RebindContextLookupContext implements LookupContext { @Override public Policy lookupPolicy(String id) { Policy result = rebindContext.getPolicy(id); if (result == null) { + result = managementContext.lookup(id, Policy.class); + } + if (result == null) { result = exceptionHandler.onDanglingPolicyRef(id); } return result; @@ -84,6 +99,9 @@ public class RebindContextLookupContext implements LookupContext { @Override public Enricher lookupEnricher(String id) { Enricher result = rebindContext.getEnricher(id); if (result == null) { + result = managementContext.lookup(id, Enricher.class); + } + if (result == null) { result = exceptionHandler.onDanglingEnricherRef(id); } return result; @@ -92,6 +110,9 @@ public class RebindContextLookupContext implements LookupContext { @Override public Feed lookupFeed(String id) { Feed result = rebindContext.getFeed(id); if (result == null) { + result = managementContext.lookup(id, Feed.class); + } + if (result == null) { result = exceptionHandler.onDanglingFeedRef(id); } return result; @@ -101,6 +122,9 @@ public class RebindContextLookupContext implements LookupContext { public CatalogItem<?, ?> lookupCatalogItem(String id) { CatalogItem<?, ?> result = rebindContext.getCatalogItem(id); if (result == null) { + result = CatalogUtils.getCatalogItemOptionalVersion(managementContext, id); + } + if (result == null) { result = exceptionHandler.onDanglingCatalogItemRef(id); } return result; http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/core/src/main/java/brooklyn/entity/rebind/RebindIteration.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/brooklyn/entity/rebind/RebindIteration.java b/core/src/main/java/brooklyn/entity/rebind/RebindIteration.java index 26ec642..b4288c7 100644 --- a/core/src/main/java/brooklyn/entity/rebind/RebindIteration.java +++ b/core/src/main/java/brooklyn/entity/rebind/RebindIteration.java @@ -25,7 +25,6 @@ import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.Callable; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -99,7 +98,6 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -144,59 +142,58 @@ Multi-phase deserialization: context entity of the Tasks.current() task. 4. Propagate the context catalog item ID to children, adjuncts if they don't have one already. */ -public class RebindIteration { +public abstract class RebindIteration { private static final Logger LOG = LoggerFactory.getLogger(RebindIteration.class); - private final RebindManagerImpl rebindManager; + protected final RebindManagerImpl rebindManager; - private final ClassLoader classLoader; - private final RebindExceptionHandler exceptionHandler; - private final ManagementNodeState mode; - private final ManagementContextInternal managementContext; - - private final Semaphore rebindActive; - private final AtomicInteger readOnlyRebindCount; - private final PersistenceActivityMetrics rebindMetrics; - private final BrooklynMementoPersister persistenceStoreAccess; + protected final ClassLoader classLoader; + protected final RebindExceptionHandler exceptionHandler; + protected final ManagementNodeState mode; + protected final ManagementContextInternal managementContext; + + protected final Semaphore rebindActive; + protected final AtomicInteger readOnlyRebindCount; + protected final PersistenceActivityMetrics rebindMetrics; + protected final BrooklynMementoPersister persistenceStoreAccess; - private final AtomicBoolean iterationStarted = new AtomicBoolean(); - private final RebindContextImpl rebindContext; - private final Reflections reflections; - private final LookupContext lookupContext; - private final BrooklynObjectInstantiator instantiator; + protected final AtomicBoolean iterationStarted = new AtomicBoolean(); + protected final RebindContextImpl rebindContext; + protected final Reflections reflections; + protected final LookupContext lookupContext; + protected final BrooklynObjectInstantiator instantiator; // populated in the course of a run // set on run start - private Stopwatch timer; + protected Stopwatch timer; /** phase is used to ensure our steps are run as we've expected, and documented (in javadoc at top). * it's worth the extra effort due to the complication and the subtleties. */ - private int phase = 0; + protected int phase = 0; // set in first phase - private BrooklynMementoRawData mementoRawData; - private BrooklynMementoManifest mementoManifest; - private Boolean overwritingMaster; - private Boolean isEmpty; + protected BrooklynMementoRawData mementoRawData; + protected BrooklynMementoManifest mementoManifest; + protected Boolean overwritingMaster; + protected Boolean isEmpty; // set later on - private BrooklynMemento memento; + protected BrooklynMemento memento; // set near the end - private List<Application> applications; + protected List<Application> applications; - public RebindIteration(RebindManagerImpl rebindManager, ManagementNodeState mode, ClassLoader classLoader, RebindExceptionHandler exceptionHandler, Semaphore rebindActive, AtomicInteger readOnlyRebindCount, PersistenceActivityMetrics rebindMetrics, BrooklynMementoPersister persistenceStoreAccess ) { - // NB: there is no particularly deep meaning in what is passed in vs what is lookup up from the RebindManager which calls us + // NB: there is no particularly deep meaning in what is passed in vs what is looked up from the RebindManager which calls us // (this is simply a refactoring of previous code to a new class) this.rebindManager = rebindManager; @@ -214,6 +211,7 @@ public class RebindIteration { rebindContext = new RebindContextImpl(exceptionHandler, classLoader); reflections = new Reflections(classLoader); lookupContext = new RebindContextLookupContext(managementContext, rebindContext, exceptionHandler); + // TODO there seems to be a lot of redundancy between RebindContext and LookupContext; do we need both? rebindContext.setLookupContext(lookupContext); instantiator = new BrooklynObjectInstantiator(classLoader, rebindContext, reflections); @@ -232,26 +230,9 @@ public class RebindIteration { return rebindContext; } - public void runFullRebind() { - runWithLock(new Callable<Void>() { - public Void call() throws Exception { - - loadManifestFiles(); - rebuildCatalog(); - instantiateLocationsAndEntities(); - instantiateMementos(); - instantiateAdjuncts(instantiator); - reconstructEverything(); - associateAdjunctsWithEntities(); - manageTheObjects(); - finishingUp(); - - return null; - } - }); - } + protected abstract void doRun() throws Exception; - protected void runWithLock(Callable<?> target) { + public void run() { if (iterationStarted.getAndSet(true)) { throw new IllegalStateException("Iteration "+this+" has already run; create a new instance for another rebind pass."); } @@ -266,7 +247,7 @@ public class RebindIteration { timer = Stopwatch.createStarted(); exceptionHandler.onStart(rebindContext); - target.call(); + doRun(); exceptionHandler.onDone(); @@ -295,52 +276,14 @@ public class RebindIteration { throw new IllegalStateException("Phase mismatch: should be phase "+targetPhase+" but is currently "+phase); } - protected void loadManifestFiles() throws Exception { - checkEnteringPhase(1); - - //The manifest contains full catalog items mementos. Reading them at this stage means that - //we don't support references to entities/locations withing tags. - - LOG.debug("Rebinding ("+mode+ - (readOnlyRebindCount.get()>Integer.MIN_VALUE ? ", iteration "+readOnlyRebindCount : "")+ - ") from "+rebindManager.getPersister().getBackingStoreDescription()+"..."); - - if (mementoRawData!=null || mementoManifest!=null) - throw new IllegalStateException("Memento data is already set"); - - mementoRawData = persistenceStoreAccess.loadMementoRawData(exceptionHandler); - mementoManifest = persistenceStoreAccess.loadMementoManifest(mementoRawData, exceptionHandler); - - determineStateFromManifestFiles(); - } - protected void determineStateFromManifestFiles() { checkContinuingPhase(1); overwritingMaster = false; isEmpty = mementoManifest.isEmpty(); - - if (mode!=ManagementNodeState.HOT_STANDBY && mode!=ManagementNodeState.HOT_BACKUP) { - if (!isEmpty) { - LOG.info("Rebinding from "+getPersister().getBackingStoreDescription()+" for "+Strings.toLowerCase(Strings.toString(mode))+" "+managementContext.getManagementNodeId()+"..."); - } else { - LOG.info("Rebind check: no existing state; will persist new items to "+getPersister().getBackingStoreDescription()); - } - - if (!managementContext.getEntityManager().getEntities().isEmpty() || !managementContext.getLocationManager().getLocations().isEmpty()) { - // this is discouraged if we were already master - Entity anEntity = Iterables.getFirst(managementContext.getEntityManager().getEntities(), null); - if (anEntity!=null && !((EntityInternal)anEntity).getManagementSupport().isReadOnly()) { - overwritingMaster = true; - LOG.warn("Rebind requested for "+mode+" node "+managementContext.getManagementNodeId()+" " - + "when it already has active state; discouraged, " - + "will likely overwrite: "+managementContext.getEntityManager().getEntities()+" and "+managementContext.getLocationManager().getLocations()+" and more"); - } - } - } } - private void rebuildCatalog() { + protected void rebuildCatalog() { // build catalog early so we can load other things checkEnteringPhase(2); @@ -415,7 +358,7 @@ public class RebindIteration { } } - private void instantiateLocationsAndEntities() { + protected void instantiateLocationsAndEntities() { checkEnteringPhase(3); @@ -454,14 +397,14 @@ public class RebindIteration { } } - private void instantiateMementos() throws IOException { + protected void instantiateMementos() throws IOException { checkEnteringPhase(4); memento = persistenceStoreAccess.loadMemento(mementoRawData, lookupContext, exceptionHandler); } - private void instantiateAdjuncts(BrooklynObjectInstantiator instantiator) { + protected void instantiateAdjuncts(BrooklynObjectInstantiator instantiator) { checkEnteringPhase(5); @@ -517,7 +460,7 @@ public class RebindIteration { } } - private void reconstructEverything() { + protected void reconstructEverything() { checkEnteringPhase(6); @@ -622,7 +565,7 @@ public class RebindIteration { } } - private void associateAdjunctsWithEntities() { + protected void associateAdjunctsWithEntities() { checkEnteringPhase(7); @@ -649,7 +592,7 @@ public class RebindIteration { } } - private void manageTheObjects() { + protected void manageTheObjects() { checkEnteringPhase(8); @@ -658,7 +601,7 @@ public class RebindIteration { Set<String> oldLocations = Sets.newLinkedHashSet(locationManager.getLocationIds()); for (Location location: rebindContext.getLocations()) { ManagementTransitionMode oldMode = locationManager.getLastManagementTransitionMode(location.getId()); - locationManager.setManagementTransitionMode(location, RebindManagerImpl.computeMode(managementContext, location, oldMode, rebindContext.isReadOnly(location)) ); + locationManager.setManagementTransitionMode(location, RebindManagerImpl.computeMode(managementContext, location, oldMode, rebindContext.isReadOnly(location), isRebindingActiveAgain()) ); if (oldMode!=null) oldLocations.remove(location.getId()); } @@ -672,12 +615,8 @@ public class RebindIteration { } } } - // destroy old - if (!oldLocations.isEmpty()) BrooklynLogging.log(LOG, overwritingMaster ? BrooklynLogging.LoggingLevel.WARN : BrooklynLogging.LoggingLevel.DEBUG, - "Destroying unused locations on rebind: "+oldLocations); - for (String oldLocationId: oldLocations) { - locationManager.unmanage(locationManager.getLocation(oldLocationId), ManagementTransitionMode.REBINDING_DESTROYED); - } + // TODO could also see about purging unreferenced locations + cleanupOldLocations(oldLocations); // Manage the top-level apps (causing everything under them to become managed) logRebindingDebug("RebindManager managing entities"); @@ -685,36 +624,41 @@ public class RebindIteration { Set<String> oldEntities = Sets.newLinkedHashSet(entityManager.getEntityIds()); for (Entity entity: rebindContext.getEntities()) { ManagementTransitionMode oldMode = entityManager.getLastManagementTransitionMode(entity.getId()); - entityManager.setManagementTransitionMode(entity, RebindManagerImpl.computeMode(managementContext,entity, oldMode, rebindContext.isReadOnly(entity)) ); + entityManager.setManagementTransitionMode(entity, RebindManagerImpl.computeMode(managementContext, entity, oldMode, rebindContext.isReadOnly(entity), isRebindingActiveAgain()) ); if (oldMode!=null) oldEntities.remove(entity.getId()); } List<Application> apps = Lists.newArrayList(); - for (String appId : memento.getApplicationIds()) { - Entity entity = rebindContext.getEntity(appId); + for (String rootId : getMementoRootEntities()) { + Entity entity = rebindContext.getEntity(rootId); if (entity == null) { // usually because of creation-failure, when not using fail-fast - exceptionHandler.onNotFound(BrooklynObjectType.ENTITY, appId); + exceptionHandler.onNotFound(BrooklynObjectType.ENTITY, rootId); } else { try { entityManager.manageRebindedRoot(entity); } catch (Exception e) { exceptionHandler.onManageFailed(BrooklynObjectType.ENTITY, entity, e); } - apps.add((Application)entity); + if (entity instanceof Application) + apps.add((Application)entity); } } - // destroy old - if (!oldLocations.isEmpty()) BrooklynLogging.log(LOG, overwritingMaster ? BrooklynLogging.LoggingLevel.WARN : BrooklynLogging.LoggingLevel.DEBUG, - "Destroying unused entities on rebind: "+oldEntities); - for (String oldEntityId: oldEntities) { - entityManager.unmanage(entityManager.getEntity(oldEntityId), ManagementTransitionMode.REBINDING_DESTROYED); - } + cleanupOldEntities(oldEntities); this.applications = apps; } - private void finishingUp() { + protected abstract boolean isRebindingActiveAgain(); + + protected Collection<String> getMementoRootEntities() { + return memento.getApplicationIds(); + } + + protected abstract void cleanupOldLocations(Set<String> oldLocations); + protected abstract void cleanupOldEntities(Set<String> oldEntities); + + protected void finishingUp() { checkContinuingPhase(8); @@ -733,10 +677,10 @@ public class RebindIteration { } // Return the top-level applications - logRebindingDebug("RebindManager complete; apps: {}", memento.getApplicationIds()); + logRebindingDebug("RebindManager complete; apps: {}", getMementoRootEntities()); } - private void noteErrors(final RebindExceptionHandler exceptionHandler, Exception primaryException) { + protected void noteErrors(final RebindExceptionHandler exceptionHandler, Exception primaryException) { List<Exception> exceptions = exceptionHandler.getExceptions(); List<String> warnings = exceptionHandler.getWarnings(); if (primaryException!=null || !exceptions.isEmpty() || !warnings.isEmpty()) { @@ -748,7 +692,7 @@ public class RebindIteration { } } - private String findCatalogItemId(ClassLoader cl, Map<String, EntityMementoManifest> entityIdToManifest, EntityMementoManifest entityManifest) { + protected String findCatalogItemId(ClassLoader cl, Map<String, EntityMementoManifest> entityIdToManifest, EntityMementoManifest entityManifest) { if (entityManifest.getCatalogItemId() != null) { return entityManifest.getCatalogItemId(); } @@ -813,19 +757,19 @@ public class RebindIteration { return null; } - private class BrooklynObjectInstantiator { + protected class BrooklynObjectInstantiator { - private final ClassLoader classLoader; - private final RebindContextImpl rebindContext; - private final Reflections reflections; + protected final ClassLoader classLoader; + protected final RebindContextImpl rebindContext; + protected final Reflections reflections; - private BrooklynObjectInstantiator(ClassLoader classLoader, RebindContextImpl rebindContext, Reflections reflections) { + protected BrooklynObjectInstantiator(ClassLoader classLoader, RebindContextImpl rebindContext, Reflections reflections) { this.classLoader = classLoader; this.rebindContext = rebindContext; this.reflections = reflections; } - private Entity newEntity(String entityId, String entityType, String catalogItemId) { + protected Entity newEntity(String entityId, String entityType, String catalogItemId) { Class<? extends Entity> entityClazz = load(Entity.class, entityType, catalogItemId, entityId); Entity entity; @@ -868,17 +812,17 @@ public class RebindIteration { return entity; } - private void setCatalogItemId(BrooklynObject item, String catalogItemId) { + protected void setCatalogItemId(BrooklynObject item, String catalogItemId) { if (catalogItemId!=null) { ((BrooklynObjectInternal)item).setCatalogItemId(catalogItemId); } } - private <T extends BrooklynObject> Class<? extends T> load(Class<T> bType, Memento memento) { + protected <T extends BrooklynObject> Class<? extends T> load(Class<T> bType, Memento memento) { return load(bType, memento.getType(), memento.getCatalogItemId(), memento.getId()); } @SuppressWarnings("unchecked") - private <T extends BrooklynObject> Class<? extends T> load(Class<T> bType, String jType, String catalogItemId, String contextSuchAsId) { + protected <T extends BrooklynObject> Class<? extends T> load(Class<T> bType, String jType, String catalogItemId, String contextSuchAsId) { checkNotNull(jType, "Type of %s (%s) must not be null", contextSuchAsId, bType.getSimpleName()); if (catalogItemId != null) { BrooklynClassLoadingContext loader = getLoadingContextFromCatalogItemId(catalogItemId, classLoader, rebindContext); @@ -909,7 +853,7 @@ public class RebindIteration { } } - private BrooklynClassLoadingContext getLoadingContextFromCatalogItemId(String catalogItemId, ClassLoader classLoader, RebindContext rebindContext) { + protected BrooklynClassLoadingContext getLoadingContextFromCatalogItemId(String catalogItemId, ClassLoader classLoader, RebindContext rebindContext) { Preconditions.checkNotNull(catalogItemId, "catalogItemId required (should not be null)"); CatalogItem<?, ?> catalogItem = rebindContext.lookup().lookupCatalogItem(catalogItemId); if (catalogItem != null) { @@ -922,7 +866,7 @@ public class RebindIteration { /** * Constructs a new location, passing to its constructor the location id and all of memento.getFlags(). */ - private Location newLocation(String locationId, String locationType) { + protected Location newLocation(String locationId, String locationType) { Class<? extends Location> locationClazz = reflections.loadClass(locationType, Location.class); if (InternalFactory.isNewStyle(locationClazz)) { @@ -953,7 +897,7 @@ public class RebindIteration { /** * Constructs a new policy, passing to its constructor the policy id and all of memento.getConfig(). */ - private Policy newPolicy(PolicyMemento memento) { + protected Policy newPolicy(PolicyMemento memento) { String id = memento.getId(); Class<? extends Policy> policyClazz = load(Policy.class, memento.getType(), memento.getCatalogItemId(), id); @@ -986,7 +930,7 @@ public class RebindIteration { /** * Constructs a new enricher, passing to its constructor the enricher id and all of memento.getConfig(). */ - private Enricher newEnricher(EnricherMemento memento) { + protected Enricher newEnricher(EnricherMemento memento) { Class<? extends Enricher> enricherClazz = load(Enricher.class, memento); String id = memento.getId(); @@ -1019,7 +963,7 @@ public class RebindIteration { /** * Constructs a new enricher, passing to its constructor the enricher id and all of memento.getConfig(). */ - private Feed newFeed(FeedMemento memento) { + protected Feed newFeed(FeedMemento memento) { Class<? extends Feed> feedClazz = load(Feed.class, memento); String id = memento.getId(); @@ -1039,7 +983,7 @@ public class RebindIteration { } @SuppressWarnings({ "rawtypes" }) - private CatalogItem<?, ?> newCatalogItem(CatalogItemMemento memento) { + protected CatalogItem<?, ?> newCatalogItem(CatalogItemMemento memento) { String id = memento.getId(); // catalog item subtypes are internal to brooklyn, not in osgi String itemType = checkNotNull(memento.getType(), "catalog item type of %s must not be null in memento", id); @@ -1047,7 +991,7 @@ public class RebindIteration { return invokeConstructor(reflections, clazz, new Object[]{}); } - private <T> T invokeConstructor(Reflections reflections, Class<T> clazz, Object[]... possibleArgs) { + protected <T> T invokeConstructor(Reflections reflections, Class<T> clazz, Object[]... possibleArgs) { for (Object[] args : possibleArgs) { try { Optional<T> v = Reflections.invokeConstructorWithArgs(clazz, args, true); @@ -1071,7 +1015,7 @@ public class RebindIteration { } /** logs at debug, except during subsequent read-only rebinds, in which it logs trace */ - private void logRebindingDebug(String message, Object... args) { + protected void logRebindingDebug(String message, Object... args) { if (shouldLogRebinding()) { LOG.debug(message, args); } else { http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/core/src/main/java/brooklyn/entity/rebind/RebindManagerImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/brooklyn/entity/rebind/RebindManagerImpl.java b/core/src/main/java/brooklyn/entity/rebind/RebindManagerImpl.java index b1a02c1..608f75c 100644 --- a/core/src/main/java/brooklyn/entity/rebind/RebindManagerImpl.java +++ b/core/src/main/java/brooklyn/entity/rebind/RebindManagerImpl.java @@ -36,7 +36,9 @@ import brooklyn.config.BrooklynServerConfig; import brooklyn.config.ConfigKey; import brooklyn.enricher.basic.AbstractEnricher; import brooklyn.entity.Application; +import brooklyn.entity.Entity; import brooklyn.entity.basic.ConfigKeys; +import brooklyn.entity.basic.Entities; import brooklyn.entity.rebind.persister.BrooklynMementoPersisterToObjectStore; import brooklyn.entity.rebind.persister.BrooklynPersistenceUtils; import brooklyn.entity.rebind.persister.BrooklynPersistenceUtils.CreateBackupMode; @@ -53,6 +55,7 @@ import brooklyn.management.internal.ManagementTransitionInfo.ManagementTransitio import brooklyn.mementos.BrooklynMementoPersister; import brooklyn.mementos.BrooklynMementoRawData; import brooklyn.mementos.TreeNode; +import brooklyn.util.collections.MutableList; import brooklyn.util.collections.MutableMap; import brooklyn.util.collections.QuorumCheck; import brooklyn.util.collections.QuorumCheck.QuorumChecks; @@ -361,6 +364,33 @@ public class RebindManagerImpl implements RebindManager { if (persistenceStoreAccess != null) persistenceStoreAccess.stop(true); } + public void testRebindNodeXXX(String ...objectsToRebindIds) { + final ClassLoader classLoader = + managementContext.getCatalog().getRootClassLoader(); + final RebindExceptionHandler exceptionHandler = + RebindExceptionHandlerImpl.builder() + .danglingRefFailureMode(danglingRefFailureMode) + .danglingRefQuorumRequiredHealthy(danglingRefsQuorumRequiredHealthy) + .rebindFailureMode(rebindFailureMode) + .addPolicyFailureMode(addPolicyFailureMode) + .loadPolicyFailureMode(loadPolicyFailureMode) + .build(); + final ManagementNodeState mode = getRebindMode(); + + ActivePartialRebindIteration iteration = new ActivePartialRebindIteration(this, mode, classLoader, exceptionHandler, + rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess); + + List<BrooklynObject> objectsToRebind = MutableList.of(); + for (String objectId: objectsToRebindIds) { + BrooklynObject obj = managementContext.lookup(objectId); + if (obj instanceof Entity) obj = Entities.deproxy((Entity)obj); + objectsToRebind.add(obj); + } + + iteration.setObjectIterator(objectsToRebind.iterator()); + iteration.run(); + } + protected ManagementNodeState getRebindMode() { if (managementContext==null) throw new IllegalStateException("Invalid "+this+": no management context"); if (!(managementContext.getHighAvailabilityManager() instanceof HighAvailabilityManagerImpl)) @@ -489,10 +519,10 @@ public class RebindManagerImpl implements RebindManager { } protected List<Application> rebindImpl(final ClassLoader classLoader, final RebindExceptionHandler exceptionHandler, ManagementNodeState mode) { - RebindIteration iteration = new RebindIteration(this, mode, classLoader, exceptionHandler, + RebindIteration iteration = new InitialFullRebindIteration(this, mode, classLoader, exceptionHandler, rebindActive, readOnlyRebindCount, rebindMetrics, persistenceStoreAccess); - iteration.runFullRebind(); + iteration.run(); if (firstRebindAppCount==null) { firstRebindAppCount = iteration.getApplications().size(); @@ -503,23 +533,41 @@ public class RebindManagerImpl implements RebindManager { return iteration.getApplications(); } + @Deprecated /** @deprecated since 0.7.0, use method with more args */ static ManagementTransitionMode computeMode(ManagementContext mgmt, BrooklynObject item, ManagementTransitionMode oldMode, boolean isNowReadOnly) { - return computeMode(mgmt, item, oldMode==null ? null : oldMode.wasReadOnly(), isNowReadOnly); + return computeMode(mgmt, item, oldMode==null ? null : oldMode.wasReadOnly(), isNowReadOnly, false); } + @Deprecated /** @deprecated since 0.7.0, use method with more args */ static ManagementTransitionMode computeMode(ManagementContext mgmt, BrooklynObject item, Boolean wasReadOnly, boolean isNowReadOnly) { + return computeMode(mgmt, item, wasReadOnly, isNowReadOnly, false); + } + + static ManagementTransitionMode computeMode(ManagementContext mgmt, BrooklynObject item, ManagementTransitionMode oldMode, boolean isNowReadOnly, boolean isRebindingActiveAgain) { + return computeMode(mgmt, item, oldMode==null ? null : oldMode.wasReadOnly(), isNowReadOnly, isRebindingActiveAgain); + } + + static ManagementTransitionMode computeMode(ManagementContext mgmt, BrooklynObject item, Boolean wasReadOnly, boolean isNowReadOnly, boolean isRebindingActiveAgain) { if (wasReadOnly==null) { // not known if (Boolean.TRUE.equals(isNowReadOnly)) return ManagementTransitionMode.REBINDING_READONLY; - else return ManagementTransitionMode.REBINDING_CREATING; + else { + // TODO is this needed? + return ManagementTransitionMode.REBINDING_CREATING; + } } else { - if (wasReadOnly && isNowReadOnly) + if (isRebindingActiveAgain) { + if (wasReadOnly || isNowReadOnly) + throw new IllegalStateException("Cannot be rebinding again to something where read-only before/after is "+wasReadOnly+"/"+isNowReadOnly); + return ManagementTransitionMode.REBINDING_ACTIVE_AGAIN; + } else if (wasReadOnly && isNowReadOnly) return ManagementTransitionMode.REBINDING_READONLY; else if (wasReadOnly) return ManagementTransitionMode.REBINDING_BECOMING_PRIMARY; else if (isNowReadOnly) return ManagementTransitionMode.REBINDING_NO_LONGER_PRIMARY; else { + if (isRebindingActiveAgain) // for the most part we handle this correctly, although there may be leaks; see HighAvailabilityManagerInMemoryTest.testLocationsStillManagedCorrectlyAfterDoublePromotion LOG.warn("Node "+(mgmt!=null ? mgmt.getManagementNodeId() : null)+" rebinding as master when already master (discouraged, may have stale references); for: "+item); return ManagementTransitionMode.REBINDING_BECOMING_PRIMARY; http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/core/src/main/java/brooklyn/entity/rebind/persister/BrooklynMementoPersisterToObjectStore.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/brooklyn/entity/rebind/persister/BrooklynMementoPersisterToObjectStore.java b/core/src/main/java/brooklyn/entity/rebind/persister/BrooklynMementoPersisterToObjectStore.java index 1d8a685..d84bf05 100644 --- a/core/src/main/java/brooklyn/entity/rebind/persister/BrooklynMementoPersisterToObjectStore.java +++ b/core/src/main/java/brooklyn/entity/rebind/persister/BrooklynMementoPersisterToObjectStore.java @@ -143,6 +143,10 @@ public class BrooklynMementoPersisterToObjectStore implements BrooklynMementoPer }})); } + public MementoSerializer<Object> getMementoSerializer() { + return getSerializerWithStandardClassLoader(); + } + protected MementoSerializer<Object> getSerializerWithStandardClassLoader() { return serializerWithStandardClassLoader; } http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java b/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java index 322de27..14fe100 100644 --- a/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java +++ b/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java @@ -443,7 +443,11 @@ public abstract class AbstractManagementContext implements ManagementContextInte Object result; result = getEntityManager().getEntity(id); if (result!=null && type.isInstance(result)) return (T)result; - // TODO policies, etc + + result = getLocationManager().getLocation(id); + if (result!=null && type.isInstance(result)) return (T)result; + + // TODO policies, enrichers, feeds return null; } http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/core/src/test/java/brooklyn/entity/rebind/RebindOnlySomeEntitiesTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/brooklyn/entity/rebind/RebindOnlySomeEntitiesTest.java b/core/src/test/java/brooklyn/entity/rebind/RebindOnlySomeEntitiesTest.java new file mode 100644 index 0000000..a7b54b7 --- /dev/null +++ b/core/src/test/java/brooklyn/entity/rebind/RebindOnlySomeEntitiesTest.java @@ -0,0 +1,44 @@ +/* + * 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 brooklyn.entity.rebind; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotSame; + +import org.testng.annotations.Test; + +import brooklyn.entity.basic.Entities; +import brooklyn.entity.proxying.EntitySpec; +import brooklyn.test.entity.TestEntity; + +public class RebindOnlySomeEntitiesTest extends RebindTestFixtureWithApp { + + @Test + public void testRebindOnlySomeSimple() throws Exception { + TestEntity c1 = origApp.addChild(EntitySpec.create(TestEntity.class)); + Entities.manage(c1); + + // XXX + newApp = rebind(); + + assertNotSame(newApp, origApp); + assertEquals(newApp.getId(), origApp.getId()); + } + +} http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java ---------------------------------------------------------------------- diff --git a/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java b/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java index 07aacd5..4f8606a 100644 --- a/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java +++ b/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java @@ -93,8 +93,8 @@ public class MySqlNodeImpl extends SoftwareProcessImpl implements MySqlNode { * So can extract lots of sensors from that. */ Maybe<SshMachineLocation> machine = Locations.findUniqueSshMachineLocation(getLocations()); - String cmd = getDriver().getStatusCmd(); if (machine.isPresent()) { + String cmd = getDriver().getStatusCmd(); feed = SshFeed.builder() .entity(this) .period(Duration.FIVE_SECONDS) http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/usage/rest-server/src/main/java/brooklyn/rest/resources/ScriptResource.java ---------------------------------------------------------------------- diff --git a/usage/rest-server/src/main/java/brooklyn/rest/resources/ScriptResource.java b/usage/rest-server/src/main/java/brooklyn/rest/resources/ScriptResource.java index 84909da..9a8cef9 100644 --- a/usage/rest-server/src/main/java/brooklyn/rest/resources/ScriptResource.java +++ b/usage/rest-server/src/main/java/brooklyn/rest/resources/ScriptResource.java @@ -75,6 +75,7 @@ public class ScriptResource extends AbstractBrooklynRestResource implements Scri if (session!=null) session.setAttribute(USER_LAST_VALUE_SESSION_ATTRIBUTE, value); } catch (Throwable t) { + log.warn("Problem in user-supplied script: "+t, t); problem = t; } finally { stdout.end(); http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/0fb0c633/usage/rest-server/src/main/java/brooklyn/rest/util/BrooklynRestResourceUtils.java ---------------------------------------------------------------------- diff --git a/usage/rest-server/src/main/java/brooklyn/rest/util/BrooklynRestResourceUtils.java b/usage/rest-server/src/main/java/brooklyn/rest/util/BrooklynRestResourceUtils.java index b2fd4a5..2dbed30 100644 --- a/usage/rest-server/src/main/java/brooklyn/rest/util/BrooklynRestResourceUtils.java +++ b/usage/rest-server/src/main/java/brooklyn/rest/util/BrooklynRestResourceUtils.java @@ -146,7 +146,7 @@ public class BrooklynRestResourceUtils { if (app==null || app.equals(findTopLevelApplication(e))) return e; throw WebResourceUtils.preconditionFailed("Application '%s' specified does not match application '%s' to which entity '%s' (%s) is associated", - application, e.getApplication().getId(), entity, e); + application, e.getApplication()==null ? null : e.getApplication().getId(), entity, e); } if (application==null) throw WebResourceUtils.notFound("Cannot find entity '%s': no known ID and application not supplied for searching", entity);