http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cfcf46df/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlockLocationImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlockLocationImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlockLocationImpl.java deleted file mode 100644 index d691550..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlockLocationImpl.java +++ /dev/null @@ -1,258 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.ggfs; - -import org.apache.ignite.*; -import org.apache.ignite.cluster.*; -import org.apache.ignite.fs.*; -import org.apache.ignite.internal.util.tostring.*; -import org.apache.ignite.internal.util.typedef.*; -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.io.*; -import java.net.*; -import java.util.*; - -/** - * File block location in the grid. - */ -public class GridGgfsBlockLocationImpl implements IgniteFsBlockLocation, Externalizable { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - private long start; - - /** */ - private long len; - - /** */ - @GridToStringInclude - private Collection<UUID> nodeIds; - - /** */ - private Collection<String> names; - - /** */ - private Collection<String> hosts; - - /** - * Empty constructor for externalizable. - */ - public GridGgfsBlockLocationImpl() { - // No-op. - } - - /** - * @param location HDFS block location. - * @param len New length. - */ - public GridGgfsBlockLocationImpl(IgniteFsBlockLocation location, long len) { - assert location != null; - - start = location.start(); - this.len = len; - - nodeIds = location.nodeIds(); - names = location.names(); - hosts = location.hosts(); - } - - /** - * @param start Start. - * @param len Length. - * @param nodes Affinity nodes. - */ - public GridGgfsBlockLocationImpl(long start, long len, Collection<ClusterNode> nodes) { - assert start >= 0; - assert len > 0; - assert nodes != null && !nodes.isEmpty(); - - this.start = start; - this.len = len; - - convertFromNodes(nodes); - } - - /** - * @return Start position. - */ - @Override public long start() { - return start; - } - - /** - * @return Length. - */ - @Override public long length() { - return len; - } - - /** - * @return Node IDs. - */ - @Override public Collection<UUID> nodeIds() { - return nodeIds; - } - - /** {@inheritDoc} */ - @Override public Collection<String> names() { - return names; - } - - /** {@inheritDoc} */ - @Override public Collection<String> hosts() { - return hosts; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = (int)(start ^ (start >>> 32)); - - res = 31 * res + (int)(len ^ (len >>> 32)); - res = 31 * res + nodeIds.hashCode(); - - return res; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (o == this) - return true; - - if (o == null || getClass() != o.getClass()) - return false; - - GridGgfsBlockLocationImpl that = (GridGgfsBlockLocationImpl)o; - - return len == that.len && start == that.start && F.eq(nodeIds, that.nodeIds) && F.eq(names, that.names) && - F.eq(hosts, that.hosts); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridGgfsBlockLocationImpl.class, this); - } - - /** - * Writes this object to data output. Note that this is not externalizable - * interface because we want to eliminate any marshaller. - * - * @param out Data output to write. - * @throws IOException If write failed. - */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - assert names != null; - assert hosts != null; - - out.writeLong(start); - out.writeLong(len); - - out.writeBoolean(nodeIds != null); - - if (nodeIds != null) { - out.writeInt(nodeIds.size()); - - for (UUID nodeId : nodeIds) - U.writeUuid(out, nodeId); - } - - out.writeInt(names.size()); - - for (String name : names) - out.writeUTF(name); - - out.writeInt(hosts.size()); - - for (String host : hosts) - out.writeUTF(host); - } - - /** - * Reads object from data input. Note we do not use externalizable interface - * to eliminate marshaller. - * - * @param in Data input. - * @throws IOException If read failed. - */ - @Override public void readExternal(ObjectInput in) throws IOException { - start = in.readLong(); - len = in.readLong(); - - int size; - - if (in.readBoolean()) { - size = in.readInt(); - - nodeIds = new ArrayList<>(size); - - for (int i = 0; i < size; i++) - nodeIds.add(U.readUuid(in)); - } - - size = in.readInt(); - - names = new ArrayList<>(size); - - for (int i = 0; i < size; i++) - names.add(in.readUTF()); - - size = in.readInt(); - - hosts = new ArrayList<>(size); - - for (int i = 0; i < size; i++) - hosts.add(in.readUTF()); - } - - /** - * Converts collection of rich nodes to block location data. - * - * @param nodes Collection of affinity nodes. - */ - private void convertFromNodes(Collection<ClusterNode> nodes) { - Collection<String> names = new LinkedHashSet<>(); - Collection<String> hosts = new LinkedHashSet<>(); - Collection<UUID> nodeIds = new ArrayList<>(nodes.size()); - - for (final ClusterNode node : nodes) { - // Normalize host names into Hadoop-expected format. - try { - Collection<InetAddress> addrs = U.toInetAddresses(node); - - for (InetAddress addr : addrs) { - if (addr.getHostName() == null) - names.add(addr.getHostAddress() + ":" + 9001); - else { - names.add(addr.getHostName() + ":" + 9001); // hostname:portNumber - hosts.add(addr.getHostName()); - } - } - } - catch (IgniteCheckedException ignored) { - names.addAll(node.addresses()); - } - - nodeIds.add(node.id()); - } - - this.nodeIds = nodeIds; - this.names = names; - this.hosts = hosts; - } -}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cfcf46df/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlocksMessage.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlocksMessage.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlocksMessage.java deleted file mode 100644 index 8fbec75..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlocksMessage.java +++ /dev/null @@ -1,257 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.ggfs; - -import org.apache.ignite.internal.*; -import org.apache.ignite.lang.*; -import org.apache.ignite.internal.util.direct.*; -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.io.*; -import java.nio.*; -import java.util.*; - -/** - * GGFS write blocks message. - */ -public class GridGgfsBlocksMessage extends GridGgfsCommunicationMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** File id. */ - private IgniteUuid fileId; - - /** Batch id */ - private long id; - - /** Blocks to store. */ - @GridDirectMap(keyType = GridGgfsBlockKey.class, valueType = byte[].class) - private Map<GridGgfsBlockKey, byte[]> blocks; - - /** - * Empty constructor required by {@link Externalizable} - */ - public GridGgfsBlocksMessage() { - // No-op. - } - - /** - * Constructor. - * - * @param fileId File ID. - * @param id Message id. - * @param blocks Blocks to put in cache. - */ - public GridGgfsBlocksMessage(IgniteUuid fileId, long id, Map<GridGgfsBlockKey, byte[]> blocks) { - this.fileId = fileId; - this.id = id; - this.blocks = blocks; - } - - /** - * @return File id. - */ - public IgniteUuid fileId() { - return fileId; - } - - /** - * @return Batch id. - */ - public long id() { - return id; - } - - /** - * @return Map of blocks to put in cache. - */ - public Map<GridGgfsBlockKey, byte[]> blocks() { - return blocks; - } - - /** {@inheritDoc} */ - @SuppressWarnings({"CloneDoesntCallSuperClone", "CloneCallsConstructors"}) - @Override public GridTcpCommunicationMessageAdapter clone() { - GridGgfsBlocksMessage _clone = new GridGgfsBlocksMessage(); - - clone0(_clone); - - return _clone; - } - - /** {@inheritDoc} */ - @Override protected void clone0(GridTcpCommunicationMessageAdapter _msg) { - super.clone0(_msg); - - GridGgfsBlocksMessage _clone = (GridGgfsBlocksMessage)_msg; - - _clone.fileId = fileId; - _clone.id = id; - _clone.blocks = blocks; - } - - /** {@inheritDoc} */ - @SuppressWarnings("all") - @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); - - if (!super.writeTo(buf)) - return false; - - if (!commState.typeWritten) { - if (!commState.putByte(directType())) - return false; - - commState.typeWritten = true; - } - - switch (commState.idx) { - case 0: - if (blocks != null) { - if (commState.it == null) { - if (!commState.putInt(blocks.size())) - return false; - - commState.it = blocks.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry<GridGgfsBlockKey, byte[]> e = (Map.Entry<GridGgfsBlockKey, byte[]>)commState.cur; - - if (!commState.keyDone) { - if (!commState.putMessage(e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putByteArray(e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(-1)) - return false; - } - - commState.idx++; - - case 1: - if (!commState.putGridUuid(fileId)) - return false; - - commState.idx++; - - case 2: - if (!commState.putLong(id)) - return false; - - commState.idx++; - - } - - return true; - } - - /** {@inheritDoc} */ - @SuppressWarnings("all") - @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); - - if (!super.readFrom(buf)) - return false; - - switch (commState.idx) { - case 0: - if (commState.readSize == -1) { - if (buf.remaining() < 4) - return false; - - commState.readSize = commState.getInt(); - } - - if (commState.readSize >= 0) { - if (blocks == null) - blocks = U.newHashMap(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - Object _val = commState.getMessage(); - - if (_val == MSG_NOT_READ) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - byte[] _val = commState.getByteArray(); - - if (_val == BYTE_ARR_NOT_READ) - return false; - - blocks.put((GridGgfsBlockKey)commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; - - commState.idx++; - - case 1: - IgniteUuid fileId0 = commState.getGridUuid(); - - if (fileId0 == GRID_UUID_NOT_READ) - return false; - - fileId = fileId0; - - commState.idx++; - - case 2: - if (buf.remaining() < 8) - return false; - - id = commState.getLong(); - - commState.idx++; - - } - - return true; - } - - /** {@inheritDoc} */ - @Override public byte directType() { - return 67; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cfcf46df/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsClientSession.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsClientSession.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsClientSession.java deleted file mode 100644 index 04ceb32..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsClientSession.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.ggfs; - -import org.jdk8.backport.*; -import org.jetbrains.annotations.*; - -import java.io.*; -import java.util.*; -import java.util.concurrent.*; - -/** - * GGFS client session. Effectively used to manage lifecycle of opened resources and close them on - * connection close. - */ -public class GridGgfsClientSession { - /** Session resources. */ - private ConcurrentMap<Long, Closeable> rsrcMap = new ConcurrentHashMap8<>(); - - /** - * Registers resource within this session. - * - * @param rsrcId Resource id. - * @param rsrc Resource to register. - */ - public boolean registerResource(long rsrcId, Closeable rsrc) { - Object old = rsrcMap.putIfAbsent(rsrcId, rsrc); - - return old == null; - } - - /** - * Gets registered resource by ID. - * - * @param rsrcId Resource ID. - * @return Resource or {@code null} if resource was not found. - */ - @Nullable public <T> T resource(Long rsrcId) { - return (T)rsrcMap.get(rsrcId); - } - - /** - * Unregister previously registered resource. - * - * @param rsrcId Resource ID. - * @param rsrc Resource to unregister. - * @return {@code True} if resource was unregistered, {@code false} if no resource - * is associated with this ID or other resource is associated with this ID. - */ - public boolean unregisterResource(Long rsrcId, Closeable rsrc) { - return rsrcMap.remove(rsrcId, rsrc); - } - - /** - * @return Registered resources iterator. - */ - public Iterator<Closeable> registeredResources() { - return rsrcMap.values().iterator(); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cfcf46df/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsCommunicationMessage.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsCommunicationMessage.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsCommunicationMessage.java deleted file mode 100644 index 40b15b6..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsCommunicationMessage.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.ggfs; - -import org.apache.ignite.*; -import org.apache.ignite.marshaller.*; -import org.apache.ignite.internal.util.direct.*; -import org.jetbrains.annotations.*; - -import java.nio.*; - -/** - * Base class for all GGFS communication messages sent between nodes. - */ -public abstract class GridGgfsCommunicationMessage extends GridTcpCommunicationMessageAdapter { - /** */ - private static final long serialVersionUID = 0L; - - /** {@inheritDoc} */ - @Override protected void clone0(GridTcpCommunicationMessageAdapter _msg) { - } - - /** - * @param marsh Marshaller. - * @throws IgniteCheckedException In case of error. - */ - public void prepareMarshal(IgniteMarshaller marsh) throws IgniteCheckedException { - // No-op. - } - - /** - * @param marsh Marshaller. - * @param ldr Class loader. - * @throws IgniteCheckedException In case of error. - */ - public void finishUnmarshal(IgniteMarshaller marsh, @Nullable ClassLoader ldr) throws IgniteCheckedException { - // No-op. - } - - /** {@inheritDoc} */ - @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); - - if (!commState.typeWritten) { - if (!commState.putByte(directType())) - return false; - - commState.typeWritten = true; - } - - return true; - } - - /** {@inheritDoc} */ - @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); - - return true; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cfcf46df/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsContext.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsContext.java deleted file mode 100644 index cea9e41..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsContext.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.ggfs; - -import org.apache.ignite.*; -import org.apache.ignite.cluster.*; -import org.apache.ignite.fs.*; -import org.apache.ignite.internal.*; -import org.apache.ignite.internal.managers.communication.*; -import org.apache.ignite.internal.util.typedef.*; -import org.jetbrains.annotations.*; - -import java.util.*; - -import static org.apache.ignite.internal.GridNodeAttributes.*; - -/** - * GGFS context holding all required components for GGFS instance. - */ -public class GridGgfsContext { - /** Kernal context. */ - private final GridKernalContext ctx; - - /** Configuration. */ - private final IgniteFsConfiguration cfg; - - /** Managers. */ - private List<GridGgfsManager> mgrs = new LinkedList<>(); - - /** Meta manager. */ - private final GridGgfsMetaManager metaMgr; - - /** Data manager. */ - private final GridGgfsDataManager dataMgr; - - /** Server manager. */ - private final GridGgfsServerManager srvMgr; - - /** Fragmentizer manager. */ - private final GridGgfsFragmentizerManager fragmentizerMgr; - - /** GGFS instance. */ - private final GridGgfsEx ggfs; - - /** - * @param ctx Kernal context. - * @param cfg GGFS configuration. - * @param metaMgr Meta manager. - * @param dataMgr Data manager. - * @param srvMgr Server manager. - * @param fragmentizerMgr Fragmentizer manager. - * @throws IgniteCheckedException If GGFs context instantiation is failed. - */ - public GridGgfsContext( - GridKernalContext ctx, - IgniteFsConfiguration cfg, - GridGgfsMetaManager metaMgr, - GridGgfsDataManager dataMgr, - GridGgfsServerManager srvMgr, - GridGgfsFragmentizerManager fragmentizerMgr - ) throws IgniteCheckedException { - this.ctx = ctx; - this.cfg = cfg; - - this.metaMgr = add(metaMgr); - this.dataMgr = add(dataMgr); - this.srvMgr = add(srvMgr); - this.fragmentizerMgr = add(fragmentizerMgr); - - ggfs = new GridGgfsImpl(this); - } - - /** - * @return GGFS instance. - */ - public GridGgfsEx ggfs() { - return ggfs; - } - - /** - * @return Kernal context. - */ - public GridKernalContext kernalContext() { - return ctx; - } - - /** - * @return GGFS configuration. - */ - public IgniteFsConfiguration configuration() { - return cfg; - } - - /** - * @return List of managers, in starting order. - */ - public List<GridGgfsManager> managers() { - return mgrs; - } - - /** - * @return Meta manager. - */ - public GridGgfsMetaManager meta() { - return metaMgr; - } - - /** - * @return Data manager. - */ - public GridGgfsDataManager data() { - return dataMgr; - } - - /** - * @return Server manager. - */ - public GridGgfsServerManager server() { - return srvMgr; - } - - /** - * @return Fragmentizer manager. - */ - public GridGgfsFragmentizerManager fragmentizer() { - return fragmentizerMgr; - } - - /** - * @param nodeId Node ID. - * @param topic Topic. - * @param msg Message. - * @param plc Policy. - * @throws IgniteCheckedException In case of error. - */ - public void send(UUID nodeId, Object topic, GridGgfsCommunicationMessage msg, GridIoPolicy plc) - throws IgniteCheckedException { - if (!kernalContext().localNodeId().equals(nodeId)) - msg.prepareMarshal(kernalContext().config().getMarshaller()); - - kernalContext().io().send(nodeId, topic, msg, plc); - } - - /** - * @param node Node. - * @param topic Topic. - * @param msg Message. - * @param plc Policy. - * @throws IgniteCheckedException In case of error. - */ - public void send(ClusterNode node, Object topic, GridGgfsCommunicationMessage msg, GridIoPolicy plc) - throws IgniteCheckedException { - if (!kernalContext().localNodeId().equals(node.id())) - msg.prepareMarshal(kernalContext().config().getMarshaller()); - - kernalContext().io().send(node, topic, msg, plc); - } - - /** - * Checks if given node is a GGFS node. - * - * @param node Node to check. - * @return {@code True} if node has GGFS with this name, {@code false} otherwise. - */ - public boolean ggfsNode(ClusterNode node) { - assert node != null; - - GridGgfsAttributes[] ggfs = node.attribute(ATTR_GGFS); - - if (ggfs != null) - for (GridGgfsAttributes attrs : ggfs) - if (F.eq(cfg.getName(), attrs.ggfsName())) - return true; - - return false; - } - - /** - * Adds manager to managers list. - * - * @param mgr Manager. - * @return Added manager. - */ - private <T extends GridGgfsManager> T add(@Nullable T mgr) { - if (mgr != null) - mgrs.add(mgr); - - return mgr; - } -}
