From 2deb535b95333b4a53aa7692eddde236824b3754 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 19:13:10 +0300 Subject: [PATCH 01/69] IGNITE-10619: implementation of file sender --- .../ignite/internal/IgniteFeatures.java | 3 + .../communication/AbstractReceiver.java | 118 +++ .../communication/AbstractTransmission.java | 104 ++ .../managers/communication/ChunkReceiver.java | 126 +++ .../managers/communication/FileReceiver.java | 152 +++ .../managers/communication/FileSender.java | 199 ++++ .../managers/communication/GridIoManager.java | 897 +++++++++++++++++- .../communication/GridIoMessageFactory.java | 17 + .../communication/SessionChannelMessage.java | 136 +++ .../communication/TransmissionHandler.java | 90 ++ .../communication/TransmissionMeta.java | 198 ++++ .../communication/TransmissionPolicy.java | 43 + .../cache/persistence/file/FileIO.java | 27 + .../persistence/file/FileIODecorator.java | 12 + .../persistence/file/RandomAccessFileIO.java | 17 + .../cache/persistence/wal/crc/FastCrc.java | 27 +- .../query/QueryIndexDescriptorImpl.java | 2 +- .../internal/util/nio/GridNioServer.java | 21 +- .../internal/util/nio/GridNioSession.java | 14 + .../internal/util/nio/GridNioSessionImpl.java | 13 + .../util/nio/GridSelectorNioSessionImpl.java | 4 +- .../tcp/TcpCommunicationSpi.java | 219 ++++- .../tcp/internal/ConnectionKey.java | 8 + .../channel/ChannelCreateRequest.java | 134 +++ .../channel/ChannelCreateResponse.java | 102 ++ .../channel/CommunicationListenerEx.java | 35 + ...GridIoManagerFileTransmissionSelfTest.java | 848 +++++++++++++++++ .../testsuites/IgniteBasicTestSuite.java | 5 +- .../processors/query/h2/opt/GridH2Table.java | 76 +- .../cache/IgniteDynamicSqlRestoreTest.java | 2 +- .../cache/index/BasicIndexTest.java | 487 +--------- 31 files changed, 3562 insertions(+), 574 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionPolicy.java create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateResponse.java create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java index e5dbca16347e5..59f6dbd033831 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java @@ -57,6 +57,9 @@ public enum IgniteFeatures { /** Command which allow to detect and cleanup garbage which could left after destroying caches in shared groups */ FIND_AND_DELETE_GARBAGE_COMMAND(8), + /** The node can communicate with other via socket channel. */ + CHANNEL_COMMUNICATION(9), + /** Distributed metastorage. */ DISTRIBUTED_METASTORAGE(11), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java new file mode 100644 index 0000000000000..fb933f27de588 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java @@ -0,0 +1,118 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.io.IOException; +import java.nio.channels.ReadableByteChannel; +import java.util.function.BooleanSupplier; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; + +import static org.apache.ignite.internal.util.IgniteUtils.assertParameter; + +/** + * Class represents a receiver of data which can be pulled from a channel by chunks of + * predefined size. Closes when a transmission of represented object ends. + */ +abstract class AbstractReceiver extends AbstractTransmission { + /** + * @param initMeta Initial file meta info. + * @param stopChecker Node stop or prcoess interrupt checker. + * @param log Ignite logger. + * @param chunkSize Size of chunks. + */ + protected AbstractReceiver( + TransmissionMeta initMeta, + BooleanSupplier stopChecker, + IgniteLogger log, + int chunkSize + ) { + super(initMeta, stopChecker, log, chunkSize); + } + + /** + * @param ch Input channel to read data from. + * @param meta Meta information about receiving file. + * @throws IOException If an io exception occurred. + * @throws IgniteCheckedException If some check failed. + */ + public void receive( + ReadableByteChannel ch, + TransmissionMeta meta + ) throws IOException, IgniteCheckedException { + assert meta != null; + + assertParameter(initMeta.name().equals(meta.name()), "Attempt to load different file " + + "[initMeta=" + initMeta + ", meta=" + meta + ']'); + + assertParameter(initMeta.offset() + transferred == meta.offset(), + "The next chunk offest is incorrect [initMeta=" + initMeta + + ", transferred=" + transferred + ", meta=" + meta + ']'); + + assertParameter(initMeta.count() == meta.count() + transferred, " The count of bytes to transfer for " + + "the next chunk is incorrect [total=" + initMeta.count() + ", transferred=" + transferred + + ", initMeta=" + initMeta + ", meta=" + meta + ']'); + + init(meta); + + // Read data from the input. + while (hasNextChunk()) { + if (Thread.currentThread().isInterrupted() || stopped()) { + throw new IgniteCheckedException("Thread has been interrupted or operation has been cancelled " + + "due to node is stopping. Channel processing has been stopped."); + } + + readChunk(ch); + } + + assert transferred == initMeta.count() : "The number of transferred bytes are not as expected " + + "[expect=" + initMeta.count() + ", actual=" + transferred + ']'; + } + + /** + * @return Current receiver state written to a {@link TransmissionMeta} instance. + */ + public TransmissionMeta state() { + return new TransmissionMeta(initMeta.name(), + initMeta.offset() + transferred, + initMeta.count(), + initMeta.params(), + policy(), + null); + } + + /** + * @return Read policy of data handling. + */ + protected TransmissionPolicy policy() { + return initMeta.policy(); + } + + /** + * @param meta Meta information about receiving file. + * @throws IgniteCheckedException If fails. + */ + protected abstract void init(TransmissionMeta meta) throws IgniteCheckedException; + + /** + * @param ch Channel to read data from. + * @throws IOException If fails. + * @throws IgniteCheckedException If fails. + */ + protected abstract void readChunk(ReadableByteChannel ch) throws IOException, IgniteCheckedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java new file mode 100644 index 0000000000000..cc40057a6d5b7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java @@ -0,0 +1,104 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.io.Closeable; +import java.nio.channels.SocketChannel; +import java.util.function.BooleanSupplier; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Class represents base object which can transmit files (read or write) by chunks of + * predefined size over an opened {@link SocketChannel}. + */ +abstract class AbstractTransmission implements Closeable { + /** Node stopping checker. */ + private final BooleanSupplier stopChecker; + + /** The size of segment for the read. */ + protected final int chunkSize; + + /** Ignite logger. */ + protected final IgniteLogger log; + + /** Initial meta with file transferred attributes. */ + protected final TransmissionMeta initMeta; + + /** The number of bytes successfully transferred druring iteration. */ + protected long transferred; + + /** + * @param initMeta Initial file meta info. + * @param stopChecker Node stop or prcoess interrupt checker. + * @param log Ignite logger. + * @param chunkSize Size of chunks. + */ + protected AbstractTransmission( + TransmissionMeta initMeta, + BooleanSupplier stopChecker, + IgniteLogger log, + int chunkSize + ) { + A.notNull(initMeta, "Initial file meta cannot be null"); + A.notNullOrEmpty(initMeta.name(), "Trasmisson name cannot be empty or null"); + A.ensure(initMeta.offset() >= 0, "File start position cannot be negative"); + A.ensure(initMeta.count() > 0, "Total number of bytes to transfer must be greater than zero"); + A.notNull(stopChecker, "Process stop checker cannot be null"); + A.ensure(chunkSize > 0, "Size of chunks to transfer data must be positive"); + + this.stopChecker = stopChecker; + this.initMeta = initMeta; + this.log = log.getLogger(AbstractTransmission.class); + this.chunkSize = chunkSize; + } + + /** + * @return Initial transmission meta. + */ + public TransmissionMeta initMeta() { + return initMeta; + } + + /** + * @return Number of bytes which has been transferred. + */ + public long transferred() { + return transferred; + } + + /** + * @return {@code true} if the transmission process should be interrupted. + */ + protected boolean stopped() { + return stopChecker.getAsBoolean(); + } + + /** + * @return {@code true} if and only if a chunked object has received all the data it expects. + */ + protected boolean hasNextChunk() { + return transferred < initMeta.count(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(AbstractTransmission.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java new file mode 100644 index 0000000000000..1fb604f908cfd --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java @@ -0,0 +1,126 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.ReadableByteChannel; +import java.util.UUID; +import java.util.function.BooleanSupplier; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Buffered chunked receiver can handle input socket channel by chunks of data and + * deliver it to an allocated {@link ByteBuffer}. + */ +class ChunkReceiver extends AbstractReceiver { + /** Chunked channel handler to process data with chunks. */ + private final IgniteThrowableConsumer hnd; + + /** The destination object to transfer data to\from. */ + private ByteBuffer buf; + + /** + * @param nodeId The remote node id receive request for transmission from. + * @param initMeta Initial file meta info. + * @param chunkSize Size of chunks. + * @param stopChecker Node stop or prcoess interrupt checker. + * @param hnd Transmission handler to process download result. + * @param log Ignite looger. + * @throws IgniteCheckedException If fails. + */ + public ChunkReceiver( + UUID nodeId, + TransmissionMeta initMeta, + int chunkSize, + BooleanSupplier stopChecker, + TransmissionHandler hnd, + IgniteLogger log + ) throws IgniteCheckedException { + super(initMeta, stopChecker, log, chunkSize); + + assert initMeta.policy() == TransmissionPolicy.CHUNK : initMeta.policy(); + + this.hnd = hnd.chunkHandler(nodeId, initMeta); + + assert this.hnd != null : "ChunkHandler must be provided by transmission handler"; + } + + /** {@inheritDoc} */ + @Override protected void init(TransmissionMeta meta) throws IgniteCheckedException { + assert meta != null; + assert buf == null; + + buf = ByteBuffer.allocate(chunkSize); + buf.order(ByteOrder.nativeOrder()); + } + + /** {@inheritDoc} */ + @Override protected void readChunk(ReadableByteChannel ch) throws IOException, IgniteCheckedException { + assert buf != null : "Buffer is used to deilver readed data to the used and cannot be null: " + this; + + buf.rewind(); + + int readed = 0; + int res; + + // Read data from input channel utill the buffer will be completely filled + // (buf.remaining() returns 0) or partitially filled buffer if it was the last chunk. + while (true) { + res = ch.read(buf); + + // Read will return -1 if remote node close connection. + if (res < 0) { + if (transferred + readed != initMeta.count()) { + throw new IOException("Input data channel reached its end, but file has not fully loaded " + + "[transferred=" + transferred + ", readed=" + readed + ", total=" + initMeta.count() + ']'); + } + + break; + } + + readed += res; + + if (readed == buf.capacity() || buf.position() == buf.capacity()) + break; + } + + if (readed == 0) + return; + + transferred += readed; + + buf.flip(); + + hnd.accept(buf); + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + buf = null; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ChunkReceiver.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java new file mode 100644 index 0000000000000..906a31962cd51 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -0,0 +1,152 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.ReadableByteChannel; +import java.nio.file.Files; +import java.util.UUID; +import java.util.function.BooleanSupplier; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.util.IgniteUtils.assertParameter; + +/** + * Class represents a chunk data receiver which is pulling data from channel vi + * {@link FileChannel#transferFrom(ReadableByteChannel, long, long)}. + */ +class FileReceiver extends AbstractReceiver { + /** The default factory to provide IO oprations over underlying file. */ + @GridToStringExclude + private final FileIOFactory fileIoFactory; + + /** Handler to notify when a file has been processed. */ + private final IgniteThrowableConsumer hnd; + + /** The abstract java representation of the chunked file. */ + private File file; + + /** The corresponding file channel to work with. */ + @GridToStringExclude + private FileIO fileIo; + + /** + * @param nodeId The remote node id receive request for transmission from. + * @param initMeta Initial file meta info. + * @param stopChecker Node stop or prcoess interrupt checker. + * @param factory Factory to produce IO interface on files. + * @param hnd Transmission handler to process download result. + * @param log Ignite logger. + * @throws IgniteCheckedException If fails. + */ + public FileReceiver( + UUID nodeId, + TransmissionMeta initMeta, + int chunkSize, + BooleanSupplier stopChecker, + FileIOFactory factory, + TransmissionHandler hnd, + IgniteLogger log + ) throws IgniteCheckedException { + super(initMeta, stopChecker, log, chunkSize); + + assert initMeta.policy() == TransmissionPolicy.FILE : initMeta.policy(); + + fileIoFactory = factory; + this.hnd = hnd.fileHandler(nodeId, initMeta); + + assert this.hnd != null : "FileHandler must be provided by transmission handler"; + + String fileAbsPath = hnd.filePath(nodeId, initMeta); + + if (fileAbsPath == null || fileAbsPath.trim().isEmpty()) + throw new IgniteCheckedException("File receiver absolute path cannot be empty or null. Receiver cannot be" + + " initialized: " + this); + + file = new File(fileAbsPath); + } + + /** {@inheritDoc} */ + @Override public void receive( + ReadableByteChannel ch, + TransmissionMeta meta + ) throws IOException, IgniteCheckedException { + super.receive(ch, meta); + + if (transferred == initMeta.count()) + hnd.accept(file); + } + + /** {@inheritDoc} */ + @Override protected void init(TransmissionMeta meta) throws IgniteCheckedException { + assert meta != null; + assert fileIo == null; + + assertParameter(meta.name().equals(initMeta.name()), "Read operation stopped. " + + "Attempt to receive a new file from channel, while the previous was not fully loaded " + + "[meta=" + meta + ", prevFile=" + initMeta.name() + ']'); + + try { + fileIo = fileIoFactory.create(file); + + fileIo.position(initMeta.offset() + transferred); + } + catch (IOException e) { + throw new IgniteCheckedException("Unable to open destination file. Receiver will will be stopped", e); + } + } + + /** {@inheritDoc} */ + @Override protected void readChunk(ReadableByteChannel ch) throws IOException { + long batchSize = Math.min(chunkSize, initMeta.count() - transferred); + + long readed = fileIo.transferFrom(ch, initMeta.offset() + transferred, batchSize); + + if (readed > 0) + transferred += readed; + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + U.closeQuiet(fileIo); + + fileIo = null; + + try { + if (transferred != initMeta.count()) + Files.delete(file.toPath()); + } + catch (IOException e) { + U.error(log, "Error deleting not fully loaded file: " + file, e); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(FileReceiver.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java new file mode 100644 index 0000000000000..5441a0dd470d3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -0,0 +1,199 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.io.File; +import java.io.IOException; +import java.io.ObjectOutput; +import java.io.Serializable; +import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; +import java.util.Map; +import java.util.function.BooleanSupplier; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.util.IgniteUtils.assertParameter; + +/** + * Class represents a data sender by chunks of predefined size. All of the chunks will be written to the + * given socket channel. It is important that for each file you are going to send a new FileSender + * instance will be created. The sender must keep its internal state of how much data already being + * transferred to send its state to remote node when reconnection required. + *

+ * The FileSender uses the zero-copy streaming algorithm, see FileChannel#transferTo for details. + * + * @see FileChannel#transferTo(long, long, WritableByteChannel) + */ +class FileSender extends AbstractTransmission { + /** Default factory to provide IO oprations over given file. */ + @GridToStringExclude + private final FileIOFactory fileIoFactory; + + /** File which will be send to remote by chunks. */ + private final File file; + + /** Corresponding file channel to work with given file. */ + @GridToStringExclude + private FileIO fileIo; + + /** + * @param file File which is going to be send by chunks. + * @param pos File offset. + * @param cnt Number of bytes to transfer. + * @param params Additional file params. + * @param stopChecker Node stop or prcoess interrupt checker. + * @param log Ignite logger. + * @param factory Factory to produce IO interface on given file. + * @param chunkSize Size of chunks. + */ + public FileSender( + File file, + long pos, + long cnt, + Map params, + BooleanSupplier stopChecker, + IgniteLogger log, + FileIOFactory factory, + int chunkSize + ) { + super(new TransmissionMeta(file.getName(), pos, cnt, params, null, null), + stopChecker, + log, + chunkSize); + + assert file != null; + + this.file = file; + fileIoFactory = factory; + } + + /** + * @param ch Output channel to write file to. + * @param oo Channel to write meta info to. + * @param connMeta Connection meta received. + * @param plc Policy of how data will be handled on remote node. + * @throws IOException If a transport exception occurred. + * @throws IgniteCheckedException If fails. + */ + public void send(WritableByteChannel ch, + ObjectOutput oo, + @Nullable TransmissionMeta connMeta, + TransmissionPolicy plc + ) throws IOException, IgniteCheckedException { + try { + // Can be not null if reconnection is going to be occurred. + if (fileIo == null) + fileIo = fileIoFactory.create(file); + } + catch (IOException e) { + // Consider this IO exeption as a user one (not the network exception) and interrupt upload process. + throw new IgniteCheckedException("Unable to initialize source file. File sender upload will be stopped", e); + } + + // If not the initial connection for the current session. + if (connMeta != null) + state(connMeta); + + // Write to remote about transission `is in active` mode. + oo.writeBoolean(false); + + // Send meta about current file to remote. + new TransmissionMeta(initMeta.name(), + initMeta.offset() + transferred, + initMeta.count() - transferred, + initMeta.params(), + plc, + null) + .writeExternal(oo); + + oo.flush(); + + while (hasNextChunk()) { + if (Thread.currentThread().isInterrupted() || stopped()) { + throw new IgniteCheckedException("Thread has been interrupted or operation has been cancelled " + + "due to node is stopping. Channel processing has been stopped."); + } + + writeChunk(ch); + } + + assert transferred == initMeta.count() : "File is not fully transferred [expect=" + initMeta.count() + + ", actual=" + transferred + ']'; + } + + /** + * @param connMeta Conneciton meta info. + * @throws IgniteCheckedException If fails. + */ + private void state(TransmissionMeta connMeta) throws IgniteCheckedException { + assert connMeta != null; + assert fileIo != null; + + // Remote note doesn't have file info. + if (connMeta.offset() < 0) + return; + + long uploadedBytes = connMeta.offset() - initMeta.offset(); + + assertParameter(initMeta.name().equals(connMeta.name()), "Attempt to transfer different file " + + "while previous is not completed [initMeta=" + initMeta + ", meta=" + connMeta + ']'); + + assertParameter(uploadedBytes >= 0, "Incorrect sync meta [offset=" + connMeta.offset() + + ", initMeta=" + initMeta + ']'); + + // No need to set new file position, if it is not changed. + if (uploadedBytes == 0) + return; + + transferred = uploadedBytes; + + U.log(log, "Update senders number of transferred bytes after reconnect: " + uploadedBytes); + } + + /** + * @param ch Channel to write data to. + * @throws IOException If fails. + */ + private void writeChunk(WritableByteChannel ch) throws IOException { + long batchSize = Math.min(chunkSize, initMeta.count() - transferred); + + long sent = fileIo.transferTo(initMeta.offset() + transferred, batchSize, ch); + + if (sent > 0) + transferred += sent; + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + U.closeQuiet(fileIo); + + fileIo = null; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(FileSender.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 83ad60f8b2c89..67ff6bff1ad60 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -17,7 +17,23 @@ package org.apache.ignite.internal.managers.communication; +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInput; +import java.io.ObjectInputStream; +import java.io.ObjectOutput; +import java.io.ObjectOutputStream; +import java.io.OutputStream; import java.io.Serializable; +import java.net.Socket; +import java.net.SocketTimeoutException; +import java.nio.channels.Channel; +import java.nio.channels.FileChannel; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SocketChannel; +import java.nio.channels.WritableByteChannel; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; @@ -49,9 +65,13 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteMessaging; +import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.internal.GridKernalContext; @@ -59,7 +79,9 @@ import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteComponentType; import org.apache.ignite.internal.IgniteDeploymentCheckedException; +import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; @@ -68,6 +90,10 @@ import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccMessage; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter; import org.apache.ignite.internal.processors.pool.PoolProcessor; import org.apache.ignite.internal.processors.security.OperationSecurityContext; @@ -78,8 +104,10 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridTuple3; import org.apache.ignite.internal.util.lang.IgnitePair; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; @@ -98,6 +126,7 @@ import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -107,6 +136,8 @@ import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE_COORDINATOR; import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER; import static org.apache.ignite.internal.GridTopic.TOPIC_IO_TEST; +import static org.apache.ignite.internal.IgniteFeatures.CHANNEL_COMMUNICATION; +import static org.apache.ignite.internal.IgniteFeatures.nodeSupports; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.DATA_STREAMER_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IDX_POOL; @@ -124,7 +155,66 @@ import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.PER_SEGMENT_Q_OPTIMIZED_RMV; /** - * Grid communication manager. + * This class represents the internal grid communication (input and output) manager + * which is placed as a layer of indirection interaction between the Ignies Kernal and Communicaiton SPI. + * Communication manager resopnsible for controlling Communication SPI which in turn is responsible + * for exchanging data between Ignites nodes. + * + *

Data exchanging

+ *

+ * Communication manager provides a rich API for data exchanging between a pair of cluster nodes. Two types + * of communication Message-based communication and File-based communication are available. + * Each of them support sending data to an arbitrary topic on the remote node (topics of {@link GridTopic} is used). + * + *

Message-based communication

+ *

+ * Ignites extension {@link Message} and {@link GridTopic} are used to provide a topic-based messaging protocol + * between cluster nodes. All of messages used for data exchanging can be devided into two general types: + * internal and user messages. + *

+ * Internal message communication is used by Ignites Kernal. Please, refer to appropriate methods below: + *

    + *
  • {@link #sendToGridTopic(ClusterNode, GridTopic, Message, byte)}
  • + *
  • {@link #sendOrderedMessage(ClusterNode, Object, Message, byte, long, boolean)}
  • + *
  • {@link #addMessageListener(Object, GridMessageListener)}
  • + *
+ *

+ * User message communication is directly exposed to the {@link IgniteMessaging} API and provides + * for user functionality for topic-based message exchanging among nodes within the cluser defined + * by {@link ClusterGroup}. Please, refer to appropriate methods below: + *

    + *
  • {@link #sendToCustomTopic(ClusterNode, Object, Message, byte)}
  • + *
  • {@link #addUserMessageListener(Object, IgniteBiPredicate, UUID)}
  • + *
+ * + *

File-based communication

+ *

+ * Sending or receiving binary data (represented by a File) over a SocketChannel is only + * possible when the build-in TcpCommunicationSpi implementation of Communication SPI is used and + * both local and remote nodes are {@link IgniteFeatures#CHANNEL_COMMUNICATION CHANNEL_COMMUNICATION} feature + * support. To ensue that the remote node satisfies all conditions the {@link #fileTransmissionSupported(ClusterNode)} + * method must be called prior to data sending. + *

+ * It is possible to receive a set of files on a particular topic (any of {@link GridTopic}) on the remote node. + * A transmission handler for desired topic must be registered prior to opening transmission sender to it. + * Use methods below are used to register handlers and open new transmissions: + *

    + *
  • {@link #addTransmissionHandler(Object, TransmissionHandler)}
  • + *
  • {@link #openTransmissionSender(UUID, Object)}
  • + *
+ *

+ * Each transmission sender opens a new transmission session to remote node prior to sending files over it. + * (see description of {@link TransmissionSender TransmissionSender} for details). The TransmissionSender + * will send all files within single session syncronously one by one. + *

+ * NOTE. It is important to call close() method or use try-with-resource + * statement to release all resources once you've done with the transmission session. This ensures that all + * resources are released on remote node in a proper way (i.e. transmission handlers are closed). + *

+ * + * @see TcpCommunicationSpi + * @see IgniteMessaging + * @see TransmissionHandler */ public class GridIoManager extends GridManagerAdapter> { /** Empty array of message factories. */ @@ -142,6 +232,42 @@ public class GridIoManager extends GridManagerAdapter CUR_PLC = new ThreadLocal<>(); + /** + * Default transfer chunk size in bytes used for sending\receiving files over a SocketChannel. + * Setting the transfer chunk size more than 1 MB is meaningless because there is + * no asymptotic benefit. What you're trying to achieve with larger transfer chunk sizes is + * fewer thread context switches, and every time we double the transfer size you have + * the context switch cost. + *

+ * Default value is {@code 256Kb}. + */ + private static final int DFLT_CHUNK_SIZE_BYTES = 256 * 1024; + + /** Map of registered handlers per each IO topic. */ + private final ConcurrentMap topicTransmissionHnds = new ConcurrentHashMap<>(); + + /** The map of already known channel read contexts by its registered topics. */ + private final ConcurrentMap rcvCtxs = new ConcurrentHashMap<>(); + + /** The map of sessions which are currently writing files and their corresponding interruption flags. */ + private final ConcurrentMap, AtomicBoolean> senderStopFlags = new ConcurrentHashMap<>(); + + /** + * Default factory to provide IO operation interface over files for futher transmission them between nodes. + * Some implementations of file senders\receivers are using the zero-copy algorithm to tranasfer bytes + * from a file to the given SocketChannel and vice-versa. So, it is necessary to produce an {@link FileIO} + * implementation based on {@link FileChannel} which is reflected in Ignite project as {@link RandomAccessFileIO}. + * + * @see FileChannel#transferTo(long, long, WritableByteChannel) + */ + private FileIOFactory fileIoFactory = new RandomAccessFileIOFactory(); + + /** The maximum number of retry attempts (read or write attempts). */ + private int retryCnt; + + /** Size of each chunk transferred over the network of data recevier or sender. */ + private int chunkSize = DFLT_CHUNK_SIZE_BYTES; + /** Listeners by topic. */ private final ConcurrentMap lsnrMap = new ConcurrentHashMap<>(); @@ -171,7 +297,7 @@ public class GridIoManager extends GridManagerAdapter> waitMap = new ConcurrentHashMap<>(); /** Communication message listener. */ - private CommunicationListener commLsnr; + private CommunicationListenerEx commLsnr; /** Grid marshaller. */ private final Marshaller marsh; @@ -198,7 +324,7 @@ public class GridIoManager extends GridManagerAdapter> ioTestMap = new AtomicReference<>(); @@ -227,6 +353,8 @@ public GridIoManager(GridKernalContext ctx) { synchronized (sysLsnrsMux) { sysLsnrs = new GridMessageListener[GridTopic.values().length]; } + + retryCnt = ctx.config().getNetworkSendRetryCount(); } /** @@ -258,7 +386,7 @@ public void resetMetrics() { @Override public void start() throws IgniteCheckedException { startSpi(); - getSpi().setListener(commLsnr = new CommunicationListener() { + getSpi().setListener(commLsnr = new CommunicationListenerEx() { @Override public void onMessage(UUID nodeId, Serializable msg, IgniteRunnable msgC) { try { onMessage0(nodeId, (GridIoMessage)msg, msgC); @@ -274,6 +402,17 @@ public void resetMetrics() { for (GridDisconnectListener lsnr : disconnectLsnrs) lsnr.onNodeDisconnected(nodeId); } + + @Override public void onChannelOpened(UUID nodeId, Serializable initMsg, Channel channel) { + try { + onChannelOpened0(nodeId, (GridIoMessage)initMsg, channel); + } + catch (ClassCastException ignored) { + U.error(log, "Communication manager received message of unknown type (will ignore): " + + initMsg.getClass().getName() + ". Most likely GridCommunicationSpi is being used directly, " + + "which is illegal - make sure to send messages only via GridProjection API."); + } + } }); ctx.addNodeAttribute(DIRECT_PROTO_VER_ATTR, DIRECT_PROTO_VER); @@ -737,6 +876,35 @@ private void format(StringBuilder b, Collection> pairs, SimpleD case EVT_NODE_LEFT: case EVT_NODE_FAILED: + busyLock.readLock().lock(); + + try { + // Stop all writer sessions. + for (Map.Entry, AtomicBoolean> writeSesEntry: senderStopFlags.entrySet()) { + if (writeSesEntry.getKey().get1().equals(nodeId)) + writeSesEntry.getValue().set(true); + } + + // Clear the context on the uploader node left. + for (Map.Entry sesEntry : rcvCtxs.entrySet()) { + ReceiverContext ioctx = sesEntry.getValue(); + + if (ioctx.nodeId.equals(nodeId)) { + ioctx.hnd.onException(nodeId, + new ClusterTopologyCheckedException("Failed to proceed download. " + + "The remote node node left the grid: " + nodeId)); + + ioctx.interrupted = true; + U.closeQuiet(ioctx.lastRcv); + + rcvCtxs.remove(sesEntry.getKey()); + } + } + } + finally { + busyLock.readLock().unlock(); + } + for (Map.Entry> e : msgSetMap.entrySet()) { ConcurrentMap map = e.getValue(); @@ -897,6 +1065,8 @@ private void format(StringBuilder b, Collection> pairs, SimpleD evtMgr.removeLocalEventListener(discoLsnr); stopping = true; + + topicTransmissionHnds.clear(); } finally { busyLock.writeLock().unlock(); @@ -911,6 +1081,50 @@ private void format(StringBuilder b, Collection> pairs, SimpleD log.debug(stopInfo()); } + /** + * @param nodeId The remote node id. + * @param channel The channel to notify listeners with. + */ + private void onChannelOpened0(UUID nodeId, GridIoMessage initMsg, Channel channel) { + Lock busyLock0 = busyLock.readLock(); + + busyLock0.lock(); + + try { + if (stopping) { + if (log.isDebugEnabled()) { + log.debug("Received communication channel create event while node stopping (will ignore) " + + "[nodeId=" + nodeId + ", msg=" + initMsg + ']'); + } + + return; + } + + if (initMsg.topic() == null) { + int topicOrd = initMsg.topicOrdinal(); + + initMsg.topic(topicOrd >= 0 ? GridTopic.fromOrdinal(topicOrd) : + U.unmarshal(marsh, initMsg.topicBytes(), U.resolveClassLoader(ctx.config()))); + } + + byte plc = initMsg.policy(); + + pools.poolForPolicy(plc).execute(new Runnable() { + @Override public void run() { + processOpenedChannel(initMsg.topic(), nodeId, (SessionChannelMessage)initMsg.message(), + (SocketChannel)channel); + } + }); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to process channel creation event due to exception " + + "[nodeId=" + nodeId + ", initMsg=" + initMsg + ']' , e); + } + finally { + busyLock0.unlock(); + } + } + /** * @param nodeId Node ID. * @param msg Message bytes. @@ -1593,6 +1807,98 @@ public boolean checkNodeLeft(UUID nodeId, IgniteCheckedException sndErr, boolean (ping && !ctx.discovery().pingNode(nodeId)); } + /** + * @param remoteId The remote note to connect to. + * @param topic The remote topic to connect to. + * @return The channel instance to communicate with remote. + */ + public TransmissionSender openTransmissionSender(UUID remoteId, Object topic) { + return new TransmissionSender(remoteId, topic); + } + + /** + * @param topic The {@link GridTopic} to register handler to. + * @param hnd Handler which will handle file upload requests. + */ + public void addTransmissionHandler(Object topic, TransmissionHandler hnd) { + TransmissionHandler hnd0 = topicTransmissionHnds.putIfAbsent(topic, hnd); + + if (hnd0 != null) + U.warn(log, "The topic already have an appropriate session handler [topic=" + topic + ']'); + } + + /** + * @param topic The topic to erase handler from. + */ + public void removeTransmissionHandler(Object topic) { + topicTransmissionHnds.remove(topic); + } + + /** + * This method must be used prior to opening a {@link TransmissionSender} by calling + * {@link #openTransmissionSender(UUID, Object)} to ensure that remote and local nodes + * are fully support direct {@link SocketChannel} connection to transfer data. + * + * @param node Remote node to check. + * @return {@code true} if file can be send over socket channel directly. + */ + public boolean fileTransmissionSupported(ClusterNode node) { + return ((CommunicationSpi)getSpi() instanceof TcpCommunicationSpi) && + nodeSupports(node, CHANNEL_COMMUNICATION); + } + + /** + * @param nodeId Destination node to connect to. + * @param topic Topic to send the request to. + * @param initMsg Channel initialization message. + * @return Established {@link Channel} to use. + * @throws IgniteCheckedException If fails. + */ + private IgniteInternalFuture openChannel( + UUID nodeId, + Object topic, + Message initMsg + ) throws IgniteCheckedException { + assert nodeId != null; + assert topic != null; + assert !locNodeId.equals(nodeId) : "Channel cannot be opened to the local node itself:" + nodeId; + assert (CommunicationSpi)getSpi() instanceof TcpCommunicationSpi : "Only TcpCommunicationSpi supports direct " + + "connections between nodes: " + getSpi().getClass(); + + ClusterNode node = ctx.discovery().node(nodeId); + + if (node == null) + throw new ClusterTopologyCheckedException("Failed to open a new channel to remote node (node left): " + nodeId); + + int topicOrd = topic instanceof GridTopic ? ((Enum)topic).ordinal() : -1; + + GridIoMessage ioMsg = createGridIoMessage(topic, + topicOrd, + initMsg, + PUBLIC_POOL, + false, + 0, + false); + + try { + if (topicOrd < 0) + ioMsg.topicBytes(U.marshal(marsh, topic)); + + return ((TcpCommunicationSpi)(CommunicationSpi)getSpi()).openChannel(node, ioMsg); + } + catch (IgniteSpiException e) { + if (e.getCause() instanceof ClusterTopologyCheckedException) + throw (ClusterTopologyCheckedException)e.getCause(); + + if (!ctx.discovery().alive(node)) + throw new ClusterTopologyCheckedException("Failed to create channel (node left): " + node.id(), e); + + throw new IgniteCheckedException("Failed to create channel (node may have left the grid or " + + "TCP connection cannot be established due to unknown issues) " + + "[node=" + node + ", topic=" + topic + ']', e); + } + } + /** * @param node Destination node. * @param topic Topic to send the message to. @@ -2312,6 +2618,258 @@ public int getOutboundMessagesQueueSize() { return getSpi().getOutboundMessagesQueueSize(); } + /** + * @param topic Topic to which the channel is created. + * @param nodeId Remote node id. + * @param initMsg Channel initialization message with additional params. + * @param channel Channel instance. + */ + private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessage initMsg, SocketChannel channel) { + ReceiverContext rcvCtx = null; + IgniteUuid newSesId = null; + ObjectInputStream in = null; + ObjectOutputStream out = null; + + try { + TransmissionHandler hnd = topicTransmissionHnds.get(topic); + + if (hnd == null) { + U.warn(log, "There is no handler for given topic. Opened channel will be closed [nodeId=" + nodeId + + ", topic=" + topic + ']'); + + return; + } + + if (initMsg == null || initMsg.sesId() == null) { + U.warn(log, "There is no initial message provied for given topic. Opened channel will be closed " + + "[nodeId=" + nodeId + ", topic=" + topic + ']'); + + return; + } + + rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(nodeId, hnd)); + + configureChannel(ctx.config(), channel); + + in = new ObjectInputStream(channel.socket().getInputStream()); + out = new ObjectOutputStream(channel.socket().getOutputStream()); + + // Do not allow multiple connection for the same session id; + if (!rcvCtx.inProgress.compareAndSet(false, true)) { + IgniteCheckedException ex; + + U.warn(log, ex = new IgniteCheckedException("Current topic is already being handled by " + + "another thread. Channel will be closed [initMsg=" + initMsg + ", channel=" + channel + + ", fromNodeId=" + nodeId + ']')); + + out.writeObject(new TransmissionMeta(ex)); + + return; + } + + if (!busyLock.readLock().tryLock()) + return; + + try { + newSesId = initMsg.sesId(); + + if (rcvCtx.sesId == null) + rcvCtx.sesId = newSesId; + else if (!rcvCtx.sesId.equals(newSesId)) { + // Attempt to receive file with new session id. Context must be reinited, + // previous session must be failed. + rcvCtx.hnd.onException(nodeId, new IgniteCheckedException("The handler has been aborted " + + "by transfer attempt with a new sessionId: " + newSesId)); + + rcvCtx = new ReceiverContext(nodeId, hnd); + rcvCtx.sesId = newSesId; + rcvCtx.inProgress.set(true); + + rcvCtxs.put(topic, rcvCtx); + } + + // Send previous context state to sync remote and local node (on manager connected). + TransmissionMeta meta = rcvCtx.lastRcv == null ? new TransmissionMeta(rcvCtx.lastSeenErr) : + rcvCtx.lastRcv.state().error(rcvCtx.lastSeenErr); + + out.writeObject(meta); + + // Begin method must be called only once. + if (!rcvCtx.sesStarted) { + rcvCtx.hnd.onBegin(nodeId); + + rcvCtx.sesStarted = true; + } + } + catch (Throwable t) { + rcvCtx.inProgress.set(false); + + throw t; + } + finally { + busyLock.readLock().unlock(); + } + + processOpenedChannel0(topic, rcvCtx, in, out, channel); + } + catch (Throwable t) { + U.error(log, "The download session cannot be finished due to unexpected error " + + "[ctx=" + rcvCtx + ", sesKey=" + newSesId + ']', t); + + if (rcvCtx != null) { + rcvCtx.lastSeenErr = new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t); + + rcvCtx.hnd.onException(nodeId, t); + } + } + finally { + U.closeQuiet(in); + U.closeQuiet(out); + U.closeQuiet(channel); + } + } + + /** + * @param topic Topic handler related to. + * @param rcvCtx Receiver read context. + * @throws Exception If processing fails. + */ + private void processOpenedChannel0( + Object topic, + ReceiverContext rcvCtx, + ObjectInputStream in, + ObjectOutputStream out, + ReadableByteChannel channel + ) throws Exception { + try { + while (true) { + if (Thread.currentThread().isInterrupted()) + throw new InterruptedException("The thread has been interrupted. Stop downloading file."); + + if (stopping) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)"); + + boolean exit = in.readBoolean(); + + if (exit) { + rcvCtx.hnd.onEnd(rcvCtx.nodeId); + + rcvCtxs.remove(topic); + + break; + } + + TransmissionMeta meta = new TransmissionMeta(); + + meta.readExternal(in); + + if (rcvCtx.lastRcv == null) { + rcvCtx.lastRcv = createReceiver(rcvCtx.nodeId, + rcvCtx.hnd, + meta, + () -> stopping || rcvCtx.interrupted); + } + + try (AbstractReceiver rcv = rcvCtx.lastRcv) { + long startTime = U.currentTimeMillis(); + + rcv.receive(channel, meta); + + // Write processing ack. + out.writeLong(rcv.transferred()); + out.flush(); + + rcvCtx.lastRcv = null; + + long downloadTime = U.currentTimeMillis() - startTime; + + U.log(log, "File has been received " + + "[name=" + rcv.initMeta().name() + ", transferred=" + rcv.transferred() + + ", time=" + (double)((downloadTime) / 1000) + " sec" + + ", retries=" + rcvCtx.retries + ", remoteId=" + rcvCtx.nodeId + ']'); + } + } + } + catch (IOException e) { + // Waiting for re-establishing connection. + U.warn(log, "Сonnection from the remote node lost. Will wait for the new one to continue file " + + "download " + "[nodeId=" + rcvCtx.nodeId + ", sesKey=" + rcvCtx.sesId + ']', e); + + rcvCtx.retries++; + + if (rcvCtx.retries == retryCnt) { + throw new IgniteCheckedException("Number of retry attempts to download file exceeded the limit. " + + "Max attempts: " + retryCnt, e); + } + } + finally { + rcvCtx.inProgress.set(false); + } + } + + /** + * Set factory to produce an FileIO abstraction over sended\received files. + * @see #fileIoFactory + * + * @param factory A new factory instance for creating {@link FileIO} + */ + void transfererFileIoFactory(FileIOFactory factory) { + fileIoFactory = factory; + } + + /** + * @param nodeId Remote node id. + * @param hnd Currnet handler instance which produces file handlers. + * @param meta Meta information about file pending to receive to create appropriate receiver. + * @param stopChecker Process interrupt checker. + * @return Chunk data recevier. + * @throws IgniteCheckedException If fails. + */ + private AbstractReceiver createReceiver( + UUID nodeId, + TransmissionHandler hnd, + TransmissionMeta meta, + BooleanSupplier stopChecker + ) throws IgniteCheckedException { + switch (meta.policy()) { + case FILE: + return new FileReceiver( + nodeId, + meta, + chunkSize, + stopChecker, + fileIoFactory, + hnd, + log); + + case CHUNK: + return new ChunkReceiver( + nodeId, + meta, + ctx.config() + .getDataStorageConfiguration() + .getPageSize(), + stopChecker, + hnd, + log); + + default: + throw new IgniteCheckedException("The type of read plc is unknown. The impelentation " + + "required: " + meta.policy()); + } + } + + /** + * @param cfg Ignite ocnfiguration to configure channel with. + * @param channel Socket channel to configure blocking mode. + * @throws IOException If fails. + */ + private static void configureChannel(IgniteConfiguration cfg, SocketChannel channel) throws IOException { + // Timeout must be enabled prior to entering the blocking mode to have effect. + channel.socket().setSoTimeout((int)cfg.getNetworkTimeout()); + channel.configureBlocking(true); + } + /** * Dumps SPI stats to diagnostic logs in case TcpCommunicationSpi is used, no-op otherwise. */ @@ -2332,6 +2890,337 @@ public void dumpStats() { X.println(">>> discoWaitMapSize: " + waitMap.size()); } + /** + * Read context holds all the information about current transfer read from channel process. + */ + private static class ReceiverContext { + /** The remote node input channel came from. */ + private final UUID nodeId; + + /** Handler currently in use flag. */ + private final AtomicBoolean inProgress = new AtomicBoolean(); + + /** Current sesssion handler. */ + @GridToStringExclude + private final TransmissionHandler hnd; + + /** Flag indicates session started. */ + private boolean sesStarted; + + /** Unique session request id. */ + private IgniteUuid sesId; + + /** The number of retry attempts of current session to wait. */ + private int retries; + + /** Last infinished downloading object. */ + private AbstractReceiver lastRcv; + + /** Last error occurred while channel is processed by registered session handler. */ + private IgniteCheckedException lastSeenErr; + + /** Flag indicates that current file handling process must be interrupted. */ + private volatile boolean interrupted; + + /** + * @param nodeId Remote node id. + * @param hnd Channel handler of current topic. + */ + public ReceiverContext(UUID nodeId, TransmissionHandler hnd) { + this.nodeId = nodeId; + this.hnd = hnd; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ReceiverContext.class, this); + } + } + + /** + * Сlass represents an implementation of transmission file writer. Each new instance of transmission sender + * will establish a new connection with unique transmission session identifier to the remote node and given + * topic (an arbitraty {@link GridTopic} can be used). + * + *

Zero-copy approach

+ *

+ * Current implementation of transmission sender is based on file zero-copy algorithm (the {@link FileSender} + * is used under the hood). It is potentially much more efficient than a simple loop that reads data from + * given file and writes it to the target socket channel. But if operating system does not support zero-copy + * file transfer, sending a file with {@link TransmissionSender} might fail or yield worse performance. + *

+ * Please, refer to http://en.wikipedia.org/wiki/Zero-copy + * or {@link FileChannel#transferTo(long, long, WritableByteChannel)} for details of such approach. + * + *

File and Chunk handlers

+ *

+ * It is possible to choose a file handler prior to sendig the file to remote node within opened transmission + * session. There are two types of handlers available: + * {@link TransmissionHandler#chunkHandler(UUID, TransmissionMeta)} and + * {@link TransmissionHandler#fileHandler(UUID, TransmissionMeta)}. You can use an appropriate + * {@link TransmissionPolicy} for {@link #send(File, long, long, Map, TransmissionPolicy)} method to switch + * between them. + * + *

Exceptions handling

+ *

+ * The transmission can have two different levels of exception which are handled differently: + *

    + *
  • transport exception(e.g. some network issues)
  • + *
  • application\handler level exception
  • + *
+ * + *

Application exceptions

+ *

+ * The transmission will be stopped immediately and wrapping IgniteCheckedExcpetion thrown in case of + * any application exception occured. + * + *

Transport exceptions

+ *

+ * All transport level exceptions of transmission file sender will require transmission to be reconnected. + * For instance, when the local node closes the socket connection in orderly way, but the file is not fully + * handled by remote node, the read operation over the same socket endpoint will return -1. Such + * result will be consideread as an IOException by handler and it will wait for reestablishing connection + * to continue file loading. + *

+ * Another example, the transmission sender gets the Connection reset by peer IOException message. + * This means that the remote node you are connected to has to reset the connection. This is usually caused by a + * high amount of traffic on the host, but may be caused by a server error or the remote node has exhausted + * system resources as well. Such IOException will be considered as reconnect required. + * + *

Timeout exceptions

+ *

+ * For read operations over the {@link InputStream} or write operation through the {@link OutputStream} + * the {@link Socket#setSoTimeout(int)} will be used and an {@link SocketTimeoutException} will be + * thrown when the timeout occured. The default value is taken from {@link IgniteConfiguration#getNetworkTimeout()}. + * + *

Release resources

+ *

+ * It is important to call close() method or use try-with-resource statement to release + * all resources once you've done with sending files. + * + * @see FileChannel#transferTo(long, long, WritableByteChannel) + */ + public class TransmissionSender implements Closeable { + /** Remote node id to connect to. */ + private final UUID remoteId; + + /** Remote topic to connect to. */ + private final Object topic; + + /** Current unique session identifier to transfer files to remote node. */ + private T2 sesKey; + + /** Instance of opened writable channel to work with. */ + private WritableByteChannel channel; + + /** Decorated with data operations socket of output channel. */ + private ObjectOutput out; + + /** Decoreated with data operations socket of input channel. */ + private ObjectInput in; + + /** + * @param remoteId The remote note to connect to. + * @param topic The remote topic to connect to. + */ + public TransmissionSender( + UUID remoteId, + Object topic + ) { + this.remoteId = remoteId; + this.topic = topic; + sesKey = new T2<>(remoteId, IgniteUuid.randomUuid()); + } + + /** + * @return The syncronization meta if case connection has been reset. + * @throws IgniteCheckedException If fails. + * @throws IOException If fails. + */ + private TransmissionMeta connect() throws IgniteCheckedException, IOException { + senderStopFlags.putIfAbsent(sesKey, new AtomicBoolean()); + + SocketChannel channel = (SocketChannel)openChannel(remoteId, + topic, + new SessionChannelMessage(sesKey.get2())) + .get(); + + configureChannel(ctx.config(), channel); + + this.channel = (WritableByteChannel)channel; + out = new ObjectOutputStream(channel.socket().getOutputStream()); + in = new ObjectInputStream(channel.socket().getInputStream()); + + TransmissionMeta syncMeta; + + try { + // Synchronize state between remote and local nodes. + syncMeta = (TransmissionMeta)in.readObject(); + } + catch (ClassNotFoundException e) { + throw new IgniteCheckedException(e); + } + + return syncMeta; + } + + /** + * @param file Source file to send to remote. + * @param params Additional transfer file description keys. + * @param plc The policy of handling data on remote. + * @throws IgniteCheckedException If fails. + */ + public void send( + File file, + Map params, + TransmissionPolicy plc + ) throws IgniteCheckedException { + send(file, 0, file.length(), params, plc); + } + + /** + * @param file Source file to send to remote. + * @param plc The policy of handling data on remote. + * @throws IgniteCheckedException If fails. + */ + public void send( + File file, + TransmissionPolicy plc + ) throws IgniteCheckedException { + send(file, 0, file.length(), new HashMap<>(), plc); + } + + /** + * @param file Source file to send to remote. + * @param offset Position to start trasfer at. + * @param cnt Number of bytes to transfer. + * @param params Additional transfer file description keys. + * @param plc The policy of handling data on remote. + * @throws IgniteCheckedException If fails. + */ + public void send( + File file, + long offset, + long cnt, + Map params, + TransmissionPolicy plc + ) throws IgniteCheckedException { + try (FileSender snd = new FileSender(file, + offset, + cnt, + params, + () -> stopping || senderStopFlags.get(sesKey).get(), + log, + fileIoFactory, + chunkSize) + ) { + if (log.isDebugEnabled()) { + log.debug("Start writing file to remote node [file=" + file.getName() + + ", rmtNodeId=" + remoteId + ", topic=" + topic + ']'); + } + + long startTime = U.currentTimeMillis(); + int retries = 0; + + while (true) { + if (Thread.currentThread().isInterrupted()) + throw new InterruptedException("The thread has been interrupted. Stop uploading file."); + + if (stopping) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)"); + + try { + TransmissionMeta connMeta = null; + + if (out == null && in == null) { + connMeta = connect(); + + assert connMeta != null; + + // Stop in case of any error occurred on remote node during file processing. + if (connMeta.error() != null) + throw connMeta.error(); + } + + snd.send(channel, out, connMeta, plc); + + // Read file received acknowledge. + long total = in.readLong(); + + assert total == snd.transferred() : "File is not fully written [expect=" + total + + ", transferred=" + snd.transferred() + ']'; + + break; + } + catch (IOException e) { + closeChannelQuiet(); + + // Re-establish the new connection to continue upload. + U.warn(log, "Connection lost while writing file to remote node and " + + "will be re-establishing [remoteId=" + remoteId + ", file=" + file.getName() + + ", sesKey=" + sesKey + ", retries=" + retries + + ", transferred=" + snd.transferred() + + ", total=" + snd.initMeta().count() + ']', e); + + retries++; + + if (retries == retryCnt) { + throw new IgniteCheckedException("The number of retry attempts to upload file exceeded " + + "the limit: " + retryCnt, e); + } + } + } + + long uploadTime = U.currentTimeMillis() - startTime; + + U.log(log, "File has been sent [name=" + file.getName() + + ", uploadTime=" + (double)((uploadTime) / 1000) + " sec, retries=" + retries + + ", transferred=" + snd.transferred() + ", remoteId=" + remoteId +']'); + + } + catch (Exception e) { + closeChannelQuiet(); + + throw new IgniteCheckedException("Exception while uploading file to the remote node. The process stopped " + + "[remoteId=" + remoteId + ", file=" + file.getName() + ", sesKey=" + sesKey + ']', e); + } + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + try { + senderStopFlags.remove(sesKey); + + if (out != null) { + U.log(log, "Close file writer session: " + sesKey); + + // Send transmission close flag. + out.writeBoolean(true); + } + } + catch (IOException e) { + U.warn(log, "An excpetion while writing close session flag occured. " + + " Session close operation has been ignored", e); + } + finally { + closeChannelQuiet(); + } + } + + /** + * Close channel and relese resources. + */ + private void closeChannelQuiet() { + U.closeQuiet(out); + U.closeQuiet(in); + U.closeQuiet(channel); + + out = null; + in = null; + channel = null; + } + } + /** * Linked chain of listeners. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java index 5d2604d966c3f..5d531e5484091 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java @@ -200,6 +200,8 @@ import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.collision.jobstealing.JobStealingRequest; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.communication.tcp.internal.channel.ChannelCreateRequest; +import org.apache.ignite.spi.communication.tcp.internal.channel.ChannelCreateResponse; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage2; import org.apache.ignite.spi.communication.tcp.messages.HandshakeWaitMessage; @@ -1161,6 +1163,21 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; + case ChannelCreateResponse.TYPE_CODE: + msg = new ChannelCreateResponse(); + + break; + + case ChannelCreateRequest.TYPE_CODE: + msg = new ChannelCreateRequest(); + + break; + + case SessionChannelMessage.TYPE_CODE: + msg = new SessionChannelMessage(); + + break; + // [-3..119] [124..129] [-23..-28] [-36..-55] - this // [120..123] - DR // [-4..-22, -30..-35] - SQL diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java new file mode 100644 index 0000000000000..2fcd73baa3374 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java @@ -0,0 +1,136 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import java.nio.channels.Channel; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * A message with additional {@link Channel} attibutes which is send on connection established and + * an appropriate channel is opened. + */ +class SessionChannelMessage implements Message { + /** Initial channel message type (value is {@code 177}). */ + public static final short TYPE_CODE = 177; + + /** Serialization version. */ + private static final long serialVersionUID = 0L; + + /** Channel session unique identifier. */ + private IgniteUuid sesId; + + /** + * No-op constructor to support {@link Externalizable} interface. + * This constructor is not meant to be used for other purposes. + */ + public SessionChannelMessage() { + // No-op. + } + + /** + * @param sesId Channel session unique identifier. + */ + public SessionChannelMessage(IgniteUuid sesId) { + this.sesId = sesId; + } + + /** + * @return The unique session id for the channel. + */ + public IgniteUuid sesId() { + return sesId; + } + + /** + * @param sesId The unique session id for the channel. + * @return {@code this} for chaining. + */ + public SessionChannelMessage sesId(IgniteUuid sesId) { + this.sesId = sesId; + return this; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeIgniteUuid("sesId", sesId)) + return false; + + writer.incrementState(); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + switch (reader.state()) { + case 0: + sesId = reader.readIgniteUuid("sesId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(SessionChannelMessage.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return TYPE_CODE; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SessionChannelMessage.class, this); + } + +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java new file mode 100644 index 0000000000000..67cf7df2372e4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java @@ -0,0 +1,90 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.io.File; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; + +/** + * Class represents a handler for the set of files considered to be transferred from the remote node. This handler + * must be registered to and appropriate topic in {@link GridIoManager} prior to opening a new transmission connection + * to this topic. + *

+ * NOTE: There is only one such handler per registered topic is allowed for the communication + * manager. Only one thread is allowed for data processing within a single topic. + * + *

TransmissionPolicy

+ *

+ * Files from the remote node can be handled of two different ways within a single established connection. + * It is up to the sender to decide how the particular file must be prccessed by the remote node. The + * TransmissionPolicy is used for such purpose. If {@link TransmissionPolicy#FILE} type is received by + * remote node the FileHandler will be picked up to process this file, the otherwise for the + * {@link TransmissionPolicy#CHUNK} the ChunkHandler will be picked up. + */ +public interface TransmissionHandler { + /** + * @param nodeId The remote node id receive request for transmission from. + */ + public void onBegin(UUID nodeId); + + /** + * The end of session transmission process. + */ + public void onEnd(UUID nodeId); + + /** + * @param err The err of fail handling process. + */ + public void onException(UUID nodeId, Throwable err); + + /** + * @param nodeId Remote node id from which request has been received. + * @param fileMeta File meta info. + * @return Absolute pathname denoting a file. + */ + public String filePath(UUID nodeId, TransmissionMeta fileMeta); + + /** + * Chunk handler represents by itself the way of input data stream processing. + * It accepts within each chunk a {@link ByteBuffer} with data from input for further processing. + * + * @param nodeId Remote node id from which request has been received. + * @param initMeta Initial handler meta info. + * @return Instance of chunk handler to process incoming data by chunks. + * @throws IgniteCheckedException If fails. + */ + public IgniteThrowableConsumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) + throws IgniteCheckedException; + + /** + * File handler represents by itself the way of input data stream processing. All the data will + * be processed under the hood using zero-copy transferring algorithm and only start file processing and + * the end of processing will be provided. + * + * @param nodeId Remote node id from which request has been received. + * @param initMeta Initial handler meta info. + * @return Intance of read handler to process incoming data like the {@link FileChannel} manner. + * @throws IgniteCheckedException If fails. + */ + public IgniteThrowableConsumer fileHandler(UUID nodeId, TransmissionMeta initMeta) + throws IgniteCheckedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java new file mode 100644 index 0000000000000..6107793b1a66b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java @@ -0,0 +1,198 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Class represents a file meta information to send to the remote node. Used to initiate a new file transfer + * process or to continue the previous unfinished from the last transmitted point. + */ +class TransmissionMeta implements Externalizable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** + * The name to associate particular meta with. + * Can be the particular file name, or an a transfer session identifier. + */ + private String name; + + /** Offest of transferred file. */ + private long offset; + + /** Number of bytes to transfer started from given offset. */ + private long cnt; + + /** Additional file params to transfer (e.g. partition id, partition name etc.). */ + private Map params; + + /** Read policy the way of how particular file will be handled. */ + private TransmissionPolicy plc; + + /** Last seen error if it has been occurred, or {@code null} the otherwise. */ + private Exception err; + + /** + * Default constructor, usually used to create meta to read channel data into. + */ + public TransmissionMeta() { + this(null); + } + + /** + * @param err Last seen error if it has been occurred, or {@code null} the otherwise. + */ + public TransmissionMeta(Exception err) { + this("", -1, -1, null, null, err); + } + + /** + * @param name The string name representation to assoticate particular meta with. + * @param offset The start position of file. + * @param cnt Number of bytes expected to transfer. + * @param params Additional transfer meta params. + * @param plc Policy of how file will be handled. + * @param err Last seen error if it has been occurred, or {@code null} the otherwise. + */ + public TransmissionMeta( + String name, + long offset, + long cnt, + Map params, + TransmissionPolicy plc, + Exception err + ) { + assert params instanceof Serializable || params == null : params.getClass(); + + this.name = name; + this.offset = offset; + this.cnt = cnt; + this.params = params; + this.plc = plc; + this.err = err; + } + + /** + * @return String representation file name. + */ + public String name() { + assert name != null; + + return name; + } + + /** + * @return Position to start channel transfer at. + */ + public long offset() { + return offset; + } + + /** + * @return Number of bytes expected to transfer. + */ + public long count() { + return cnt; + } + + /** + * @return The map of additional keys. + */ + public Map params() { + return params; + } + + /** + * @return File read way policy {@link TransmissionPolicy}. + */ + public TransmissionPolicy policy() { + return plc; + } + + /** + * @param err An exception instance if it has been previously occurred. + */ + public TransmissionMeta error(Exception err) { + this.err = err; + + return this; + } + + /** + * @return An exception instance if it has been previously occurred. + */ + public Exception error() { + return err; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeUTF(name); + out.writeLong(offset); + out.writeLong(cnt); + out.writeObject(params); + out.writeObject(plc); + out.writeObject(err); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + name = in.readUTF(); + offset = in.readLong(); + cnt = in.readLong(); + params = (Map)in.readObject(); + plc = (TransmissionPolicy)in.readObject(); + err = (Exception)in.readObject(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + TransmissionMeta meta = (TransmissionMeta)o; + + return offset == meta.offset && + cnt == meta.cnt && + name.equals(meta.name) && + Objects.equals(params, meta.params) && + plc == meta.plc && + Objects.equals(err, meta.err); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(name, offset, cnt, params, plc, err); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(TransmissionMeta.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionPolicy.java new file mode 100644 index 0000000000000..d4b017a9591c2 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionPolicy.java @@ -0,0 +1,43 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.util.UUID; + +/** + * Class represents ways of data handling for a file ready to be sent though an opened transmission sender session. + * It is necessary to choose which type of handler will be used and how file should be handled prior to sending file + * to the remote node. + * + * @see GridIoManager.TransmissionSender + */ +public enum TransmissionPolicy { + /** + * A file which is considered to be sent though TransmissionSenders session will use + * the {@link TransmissionHandler#fileHandler(UUID, TransmissionMeta)} of {@link TransmissionHandler} + * to handle transmitted binary data. + */ + FILE, + + /** + * A file which is considered to be sent though TransmissionSenders session will use + * the {@link TransmissionHandler#chunkHandler(UUID, TransmissionMeta)} of {@link TransmissionHandler} + * to handle transmitted binary data. This file will be processed by chunks of handlers defined size. + */ + CHUNK +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java index 546d1a7a16a30..3ecdd22f962c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.MappedByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; /** * Interface to perform file I/O operations. @@ -272,4 +274,29 @@ public interface FileIO extends AutoCloseable { * @see #punchHole */ long getSparseSize(); + + /** + * This method will transfers the content of file to the specified channel. This is a synchronous + * operation, so performing it on asynchronous channels makes no sense and not provied. + * + * @param position The relative offset of the file where the transfer begins from. + * @param count The number of bytes to be transferred. + * @param target Destination channel of the transfer. + * @return Count of bytes which was successfully transferred. + * @throws IOException If fails. + */ + public default long transferTo(long position, long count, WritableByteChannel target) throws IOException { + throw new UnsupportedOperationException(); + } + + /** + * @param src The source channel. + * @param position The position within the file at which the transfer is to begin. + * @param count The maximum number of bytes to be transferred. + * @return The number of bytes, possibly zero, that were actually transferred. + * @throws IOException If fails. + */ + public default long transferFrom(ReadableByteChannel src, long position, long count) throws IOException { + throw new UnsupportedOperationException(); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java index c615a34e6abac..dfefd29ff4f83 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.MappedByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; /** * Decorator class for File I/O @@ -120,4 +122,14 @@ public FileIODecorator(FileIO delegate) { @Override public void close() throws IOException { delegate.close(); } + + /** {@inheritDoc} */ + @Override public long transferTo(long position, long count, WritableByteChannel target) throws IOException { + return delegate.transferTo(position, count, target); + } + + /** {@inheritDoc} */ + @Override public long transferFrom(ReadableByteChannel src, long position, long count) throws IOException { + return delegate.transferFrom(src, position, count); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java index c6922bc09b2d5..668960964e4e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java @@ -23,6 +23,8 @@ import java.nio.ByteBuffer; import java.nio.MappedByteBuffer; import java.nio.channels.FileChannel; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; import java.nio.file.OpenOption; import org.apache.ignite.internal.processors.compress.FileSystemUtils; import org.apache.ignite.internal.util.typedef.internal.U; @@ -147,4 +149,19 @@ private static int getNativeFileDescriptor(FileChannel ch) { @Override public void force() throws IOException { force(false); } + + /** {@inheritDoc} */ + @Override public long transferTo(long position, long count, WritableByteChannel target) throws IOException { + return ch.transferTo(position, count, target); + } + + /** {@inheritDoc} */ + @Override public long transferFrom(ReadableByteChannel src, long position, long count) throws IOException { + long written = ch.transferFrom(src, position, count); + + if (written > 0) + position(position + written); + + return written; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java index 0dcbafdb9c978..7cbaadb8ed773 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java @@ -17,8 +17,13 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.crc; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; import java.util.zip.CRC32; +import java.util.zip.CheckedInputStream; /** * This CRC calculation implementation workf much faster then {@link PureJavaCrc32} @@ -80,6 +85,26 @@ public static int calcCrc(ByteBuffer buf, int len) { return res; } + /** + * @param file A file to calculate checksum over it. + * @return CRC32 checksum. + * @throws IOException If fails. + */ + public static int calcCrc(File file) throws IOException { + assert !file.isDirectory() : "CRC32 can't be calculated over directories"; + + CRC32 algo = new CRC32(); + + try (InputStream in = new CheckedInputStream(new FileInputStream(file), algo)) { + byte[] buf = new byte[1024]; + + while (in.read(buf) != -1) + ; + } + + return ~(int)algo.getValue(); + } + /** * @param crcAlgo CRC algorithm. * @param buf Input buffer. @@ -96,6 +121,6 @@ private static int calcCrc(CRC32 crcAlgo, ByteBuffer buf, int len) { buf.limit(initLimit); - return (int)crcAlgo.getValue() ^ 0xFFFFFFFF; + return ~(int)crcAlgo.getValue(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java index d8b972cdbbec3..1b85af579f8c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java @@ -127,7 +127,7 @@ public QueryIndexDescriptorImpl addField(String field, int orderNum, boolean des if (descending) { if (descendings == null) - descendings = new HashSet<>(); + descendings = new HashSet<>(); descendings.add(field); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java index 4584c878eea76..4d6dfb84aaad5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java @@ -2683,14 +2683,28 @@ private void closeKey(SelectionKey key) { } } + /** + * @param ses Session to be closed. + * @param e Exception to be passed to the listener, if any. + * @return {@code True} if this call closed the ses. + */ + protected boolean close(final GridSelectorNioSessionImpl ses, @Nullable final IgniteCheckedException e) { + return close(ses, e, ses.closeSocketOnSessionClose()); + } + /** * Closes the session and all associated resources, then notifies the listener. * * @param ses Session to be closed. * @param e Exception to be passed to the listener, if any. + * @param closeSock If {@code True} the channel will be closed. * @return {@code True} if this call closed the ses. */ - protected boolean close(final GridSelectorNioSessionImpl ses, @Nullable final IgniteCheckedException e) { + protected boolean close( + final GridSelectorNioSessionImpl ses, + @Nullable final IgniteCheckedException e, + boolean closeSock + ) { if (e != null) { // Print stack trace only if has runtime exception in it's cause. if (e.hasCause(IOException.class)) @@ -2714,7 +2728,10 @@ protected boolean close(final GridSelectorNioSessionImpl ses, @Nullable final Ig GridUnsafe.cleanDirectBuffer(ses.readBuffer()); } - closeKey(ses.key()); + if (closeSock) + closeKey(ses.key()); + else + ses.key().cancel(); // Unbind socket to the current SelectionKey. if (e != null) filterChain.onExceptionCaught(ses, e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java index 12b9b40b10c34..8d4af8bcb1772 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java @@ -70,6 +70,20 @@ public interface GridNioSession { */ public long closeTime(); + /** + * @return {@code True} to close SocketChannel on current session close occured. + */ + public default boolean closeSocketOnSessionClose() { + return true; + } + + /** + * @param closeSocket {@code False} remain SocketChannel open on session close. + */ + public default void closeSocketOnSessionClose(boolean closeSocket) { + throw new UnsupportedOperationException(); + } + /** * Returns the time when last read activity was performed on this session. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java index 51cb5581d5d5a..ce951b18bfe9a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java @@ -80,6 +80,9 @@ public class GridNioSessionImpl implements GridNioSession { /** For debug purposes. */ private volatile boolean markedForClose; + /** Close channel on session #close() called. */ + private volatile boolean closeSocket = true; + /** * @param filterChain Chain. * @param locAddr Local address. @@ -324,6 +327,16 @@ public boolean setClosed() { return closeTime.compareAndSet(0, U.currentTimeMillis()); } + /** {@inheritDoc} */ + @Override public boolean closeSocketOnSessionClose() { + return closeSocket; + } + + /** {@inheritDoc} */ + @Override public void closeSocketOnSessionClose(boolean closeSocket) { + this.closeSocket = closeSocket; + } + /** * @return {@code True} if this session was closed. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java index 2739299416f3f..11fc6fc83a3b2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java @@ -39,7 +39,7 @@ * Note that this implementation requires non-null values for local and remote * socket addresses. */ -class GridSelectorNioSessionImpl extends GridNioSessionImpl implements GridNioKeyAttachment { +public class GridSelectorNioSessionImpl extends GridNioSessionImpl implements GridNioKeyAttachment { /** Pending write requests. */ private final FastSizeDeque queue = new FastSizeDeque<>(new ConcurrentLinkedDeque<>()); @@ -176,7 +176,7 @@ public ByteBuffer readBuffer() { /** * @return Registered selection key for this session. */ - SelectionKey key() { + public SelectionKey key() { return key; } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 0c990517e7dd1..311242e2d784c 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -27,9 +27,9 @@ import java.net.SocketTimeoutException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.channels.Channel; import java.nio.channels.SelectableChannel; import java.nio.channels.SocketChannel; -import java.nio.channels.spi.AbstractInterruptibleChannel; import java.util.ArrayList; import java.util.Arrays; import java.util.BitSet; @@ -43,6 +43,7 @@ import java.util.StringJoiner; import java.util.UUID; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; @@ -98,6 +99,7 @@ import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter; import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey; +import org.apache.ignite.internal.util.nio.GridSelectorNioSessionImpl; import org.apache.ignite.internal.util.nio.GridShmemCommunicationClient; import org.apache.ignite.internal.util.nio.GridTcpNioCommunicationClient; import org.apache.ignite.internal.util.nio.ssl.BlockingSslHandler; @@ -147,6 +149,9 @@ import org.apache.ignite.spi.communication.tcp.internal.HandshakeException; import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture; import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationNodeConnectionCheckFuture; +import org.apache.ignite.spi.communication.tcp.internal.channel.ChannelCreateRequest; +import org.apache.ignite.spi.communication.tcp.internal.channel.ChannelCreateResponse; +import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage2; import org.apache.ignite.spi.communication.tcp.messages.HandshakeWaitMessage; @@ -161,6 +166,8 @@ import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; +import static org.apache.ignite.internal.IgniteFeatures.CHANNEL_COMMUNICATION; +import static org.apache.ignite.internal.IgniteFeatures.nodeSupports; import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META; import static org.apache.ignite.plugin.extensions.communication.Message.DIRECT_TYPE_SIZE; import static org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture.SES_FUT_META; @@ -373,6 +380,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati /** Default connections per node. */ public static final int DFLT_CONN_PER_NODE = 1; + /** Maximum {@link GridNioSession} connections per node. */ + public static final int MAX_CONN_PER_NODE = 1024; + + /** Maximum {@link Channel} connections per node. */ + public static final int MAX_CHANNEL_CONN_PER_NODE = 256; + /** No-op runnable. */ private static final IgniteRunnable NOOP = new IgniteRunnable() { @Override public void run() { @@ -398,6 +411,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati /** */ private ConnectionPolicy connPlc = new FirstConnectionPolicy(); + /** */ + private ConnectionPolicy sockConnPlc = new ChannelRandomConnectionPolicy(); + /** */ private boolean enableForcibleNodeKill = IgniteSystemProperties .getBoolean(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); @@ -739,6 +755,77 @@ private void closeStaleConnections(ConnectionKey connKey) { } } + private void handleChannelCreateResponse(GridSelectorNioSessionImpl ses, ConnectionKey connKey) { + GridFutureAdapter reqFut = channelReqs.remove(connKey); + + if (reqFut == null) { + U.error(log, "There is not corresponding channel request to the received channel create " + + "response message. Message will be ignored [remoteId=" + connKey.nodeId() + + ", idx=" + connKey.connectionIndex() + ']'); + + return; + } + + ses.closeSocketOnSessionClose(false); + + ses.close().listen(f -> { + try { + f.get(); // Exception not ocurred. + + cleanupLocalNodeRecoveryDescriptor(connKey); + + SelectableChannel nioChannel = ses.key().channel(); + + reqFut.onDone(nioChannel); + } + catch (IgniteCheckedException e) { + reqFut.onDone(e); + } + }); + } + + private void handleChannelCreateRequest( + GridSelectorNioSessionImpl ses, + ConnectionKey connKey, + ChannelCreateRequest msg + ) { + ses.send(new ChannelCreateResponse()) + .listen(sendFut -> { + try { + sendFut.get(); // Exception not ocurred. + + ses.closeSocketOnSessionClose(false); + + // Close session and send response. + ses.close().listen(closeFut -> { + try { + closeFut.get(); // Exception not ocurred. + + cleanupLocalNodeRecoveryDescriptor(connKey); + + SelectableChannel channel = ses.key().channel(); + + notifyChannelEvtListener(connKey.nodeId(), channel, msg.message()); + } + catch (IgniteCheckedException e) { + U.error(log, "Nio session has not been properly closed " + + "[nodeId=" + connKey.nodeId() + ", idx=" + connKey.connectionIndex() + ']', e); + + ses.closeSocketOnSessionClose(true); + U.closeQuiet(ses.key().channel()); + } + }); + } + catch (IgniteCheckedException e) { + U.error(log, "Fail to send channel creation response to the remote node. " + + "Session will be closed [nodeId=" + connKey.nodeId() + + ", idx=" + connKey.connectionIndex() + ']', e); + + ses.close(); + } + }); + } + @Override public void onMessage(final GridNioSession ses, Message msg) { ConnectionKey connKey = ses.meta(CONN_IDX_META); @@ -839,7 +926,22 @@ else if (connKey.dummy()) { else c = NOOP; - notifyListener(connKey.nodeId(), msg, c); + if (msg instanceof ChannelCreateRequest) { + handleChannelCreateRequest((GridSelectorNioSessionImpl)ses, connKey, + (ChannelCreateRequest)msg); + + if (c != null) + c.run(); + } + else if (msg instanceof ChannelCreateResponse) { + // msg will be ignored. + handleChannelCreateResponse((GridSelectorNioSessionImpl)ses, connKey); + + if (c != null) + c.run(); + } + else + notifyListener(connKey.nodeId(), msg, c); } } @@ -1179,6 +1281,9 @@ class ConnectClosure implements IgniteInClosure { /** Clients. */ private final ConcurrentMap clients = GridConcurrentFactory.newMap(); + /** Java NIO channels. */ + private final ConcurrentMap> channelReqs = new ConcurrentHashMap<>(); + /** SPI listener. */ private volatile CommunicationListener lsnr; @@ -2122,7 +2227,7 @@ private void dumpInfo(StringBuilder sb, UUID dstNodeId) { assertParameter(shmemPort > 0 || shmemPort == -1, "shmemPort > 0 || shmemPort == -1"); assertParameter(selectorsCnt > 0, "selectorsCnt > 0"); assertParameter(connectionsPerNode > 0, "connectionsPerNode > 0"); - assertParameter(connectionsPerNode <= 1024, "connectionsPerNode <= 1024"); + assertParameter(connectionsPerNode <= MAX_CONN_PER_NODE, "connectionsPerNode <= 1024"); if (!failureDetectionTimeoutEnabled()) { assertParameter(reconCnt > 0, "reconnectCnt > 0"); @@ -3955,6 +4060,21 @@ else if (log.isDebugEnabled()) "is node stopping?) [senderNodeId=" + sndId + ", msg=" + msg + ']'); } + /** + * @param nodeId The remote node id. + * @param channel The configured channel to notify listeners with. + * @param initMsg Channel initialization message with additional channel params. + */ + private void notifyChannelEvtListener(UUID nodeId, Channel channel, Message initMsg) { + if (log.isDebugEnabled()) + log.debug("Notify corresponding listeners due to the new channel opened: " + channel); + + CommunicationListener lsnr0 = lsnr; + + if (lsnr0 instanceof CommunicationListenerEx) + ((CommunicationListenerEx)lsnr0).onChannelOpened(nodeId, initMsg, channel); + } + /** * @param target Target buffer to append to. * @param src Source buffer to get data. @@ -4040,6 +4160,20 @@ private boolean usePairedConnections(ClusterNode node) { return false; } + /** + * @param key The connection key to cleanup descriptors on local node. + */ + private void cleanupLocalNodeRecoveryDescriptor(ConnectionKey key) { + ClusterNode node = getLocalNode(); + + if (usePairedConnections(node)){ + inRecDescs.remove(key); + outRecDescs.remove(key); + } + else + recoveryDescs.remove(key); + } + /** * @param recoveryDescs Descriptors map. * @param pairedConnections {@code True} if in/out connections pair is used for communication with node. @@ -4232,6 +4366,75 @@ private static WorkersRegistry getWorkersRegistry(Ignite ignite) { return ignite instanceof IgniteEx ? ((IgniteEx)ignite).context().workersRegistry() : null; } + /** + * @param remote Destination cluster node to communicate with. + * @param initMsg Configuration channel attributes wrapped into the message. + * @return The future, which will be finished on channel ready. + * @throws IgniteSpiException If fails. + */ + public IgniteInternalFuture openChannel( + ClusterNode remote, + Message initMsg + ) throws IgniteSpiException { + assert !remote.isLocal() : remote; + assert initMsg != null; + assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direct connection over socket channel " + + "[nodeId=" + remote.id() + ']'; + + ConnectionKey key = new ConnectionKey(remote.id(), sockConnPlc.connectionIndex()); + + GridFutureAdapter result; + + connectGate.enter(); + + try { + if (channelReqs.get(key) != null) { + throw new IgniteSpiException("The channel connection cannot be established to remote node. " + + "Connection key already in use [key=" + key + ']'); + } + + GridNioSession ses = createNioSession(remote, key.connectionIndex()); + + assert ses != null : "Session must be established [remoteId=" + remote.id() + ", key=" + key + ']'; + + final GridNioSession finalSes = ses; + + channelReqs.put(key, result = new GridFutureAdapter<>()); + + // Send configuration message over the created session. + ses.send(new ChannelCreateRequest(initMsg)) + .listen(f -> { + if (f.error() == null) { + addTimeoutObject(new IgniteSpiTimeoutObject() { + @Override public IgniteUuid id() { + return IgniteUuid.randomUuid(); + } + + @Override public long endTime() { + return U.currentTimeMillis() + DFLT_CONN_TIMEOUT; + } + + @Override public void onTimeout() { + // Close session if request not complete yet. + if (result.onDone(handshakeTimeoutException())) + finalSes.close(); + } + }); + } + else + result.onDone(f.error()); + }); + + return result; + } + catch (IgniteCheckedException e) { + throw new IgniteSpiException("Unable to create new channel connection to the remote node: " + remote, e); + } + finally { + connectGate.leave(); + } + } + /** * */ @@ -4679,7 +4882,7 @@ boolean cancel() { if (obj instanceof GridCommunicationClient) ((GridCommunicationClient)obj).forceClose(); else - U.closeQuiet((AbstractInterruptibleChannel)obj); + U.closeQuiet((AutoCloseable)obj); } } @@ -4909,6 +5112,14 @@ private class RoundRobinConnectionPolicy implements ConnectionPolicy { } } + /** */ + private static class ChannelRandomConnectionPolicy implements ConnectionPolicy { + /** {@inheritDoc} */ + @Override public int connectionIndex() { + return MAX_CONN_PER_NODE + (int)(U.safeAbs(Thread.currentThread().getId() % MAX_CHANNEL_CONN_PER_NODE)); + } + } + /** * MBean implementation for TcpCommunicationSpi. */ diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java index 0559df7892459..a107c876b26e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java @@ -37,6 +37,14 @@ public class ConnectionKey { /** */ private final boolean dummy; + /** + * @param nodeId Node ID. Should be not null. + * @param idx Connection index. + */ + public ConnectionKey(@NotNull UUID nodeId, int idx) { + this(nodeId, idx, -1, false); + } + /** * Creates ConnectionKey with false value of dummy flag. * diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateRequest.java new file mode 100644 index 0000000000000..2739030ad926e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateRequest.java @@ -0,0 +1,134 @@ +/* + * 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.apache.ignite.spi.communication.tcp.internal.channel; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import java.nio.channels.Channel; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * Message requesting to creation of {@link Channel}. + */ +public class ChannelCreateRequest implements Message { + /** Request message type (value is {@code 176}). */ + public static final short TYPE_CODE = 176; + + /** Serialization version. */ + private static final long serialVersionUID = 0L; + + /** Initialization channel message which contains channel params. */ + private Message msg; + + /** + * No-op constructor to support {@link Externalizable} interface. + * This constructor is not meant to be used for other purposes. + */ + public ChannelCreateRequest() { + // No-op. + } + + /** + * @param msg Initial channel message, containing channel attributes. + */ + public ChannelCreateRequest(Message msg) { + this.msg = msg; + } + + /** + * @return Channel initialization message. + */ + public Message message() { + return msg; + } + + /** + * @param msg Channel initialization message. + * @return {@code this} for chaining. + */ + public ChannelCreateRequest message(Message msg) { + this.msg = msg; + return this; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeMessage("msg", msg)) + return false; + + writer.incrementState(); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + switch (reader.state()) { + case 0: + msg = reader.readMessage("msg"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(ChannelCreateRequest.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return TYPE_CODE; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ChannelCreateRequest.class, this); + } + +} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateResponse.java new file mode 100644 index 0000000000000..356c5f0eeccc2 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateResponse.java @@ -0,0 +1,102 @@ +/* + * 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.apache.ignite.spi.communication.tcp.internal.channel; + +import java.nio.ByteBuffer; +import java.nio.channels.Channel; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * Message response to creation of {@link Channel}. + */ +public class ChannelCreateResponse implements Message { + /** Request message type (value is {@code 175}). */ + public static final short TYPE_CODE = 175; + + /** Serialization version. */ + private static final long serialVersionUID = 0L; + + /** Ack processing byte (used as message content). */ + private byte ack; + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeByte("ack", ack)) + return false; + + writer.incrementState(); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + switch (reader.state()) { + case 0: + ack = reader.readByte("ack"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(ChannelCreateResponse.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return TYPE_CODE; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ChannelCreateResponse.class, this); + } + +} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java new file mode 100644 index 0000000000000..46389c8098209 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java @@ -0,0 +1,35 @@ +/* + * 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.apache.ignite.spi.communication.tcp.internal.channel; + +import java.io.Serializable; +import java.nio.channels.Channel; +import java.util.UUID; +import org.apache.ignite.spi.communication.CommunicationListener; + +/** + * Extended communication SPI listener to provide {@link Channel} opened events. + */ +public interface CommunicationListenerEx extends CommunicationListener { + /** + * @param nodeId Remote node id. + * @param initMsg Init channel message. + * @param channel Locally created channel endpoint. + */ + public void onChannelOpened(UUID nodeId, T initMsg, Channel channel); +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java new file mode 100644 index 0000000000000..a07a7076bffc8 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -0,0 +1,848 @@ +/* + * 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.apache.ignite.internal.managers.communication; + +import java.io.File; +import java.io.FilenameFilter; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.nio.file.OpenOption; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.LongAdder; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteException; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridTopic; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; +import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX; +import static org.apache.ignite.internal.util.IgniteUtils.fileCount; + +/** + * Test file transmission mamanger operations. + */ +public class GridIoManagerFileTransmissionSelfTest extends GridCommonAbstractTest { + /** Number of cache keys to generate. */ + private static final long CACHE_SIZE = 50_000L; + + /** Temporary directory to store files. */ + private static final String TEMP_FILES_DIR = "ctmp"; + + /** Factory to produce IO interfaces over files to transmit. */ + private static final FileIOFactory IO_FACTORY = new RandomAccessFileIOFactory(); + + /** The topic to send files to. */ + private static Object topic; + + /** File filter. */ + private static FilenameFilter fileBinFilter; + + /** Locally used fileIo to interact with output file. */ + private final FileIO[] fileIo = new FileIO[1]; + + /** The temporary directory to store files. */ + private File tempStore; + + /** + * @throws Exception If fails. + */ + @BeforeClass + public static void beforeAll() throws Exception { + topic = GridTopic.TOPIC_CACHE.topic("test", 0); + + fileBinFilter = new FilenameFilter() { + @Override public boolean accept(File dir, String name) { + return name.endsWith(FILE_SUFFIX); + } + }; + } + + /** + * @throws Exception if failed. + */ + @Before + public void before() throws Exception { + cleanPersistenceDir(); + + tempStore = U.resolveWorkDirectory(U.defaultWorkDirectory(), TEMP_FILES_DIR, true); + } + + /** + * @throws Exception if failed. + */ + @After + public void after() throws Exception { + stopAllGrids(); + + U.closeQuiet(fileIo[0]); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + return super.getConfiguration(igniteInstanceName) + .setDataStorageConfiguration(new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setPersistenceEnabled(true) + .setMaxSize(500L * 1024 * 1024))) + .setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME)); + } + + /** + * Transmit all cache partition to particular topic on the remote node. + * + * @throws Exception If fails. + */ + @Test + public void testFileHandlerBase() throws Exception { + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + snd.cluster().active(true); + + addCacheData(snd, DEFAULT_CACHE_NAME); + + awaitPartitionMapExchange(); + + Map fileSizes = new HashMap<>(); + Map fileCrcs = new HashMap<>(); + Map fileParams = new HashMap<>(); + + assertTrue(snd.context().io().fileTransmissionSupported(rcv.localNode())); + + rcv.context().io().addTransmissionHandler(topic, new TransmissionHandlerAdapter() { + @Override public void onBegin(UUID nodeId) { + assertEquals(snd.localNode().id(), nodeId); + } + + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { + return new File(tempStore, fileMeta.name()).getAbsolutePath(); + } + + @Override public IgniteThrowableConsumer fileHandler(UUID nodeId, TransmissionMeta initMeta) { + return new IgniteThrowableConsumer() { + + @Override public void accept(File file) { + assertTrue(fileSizes.containsKey(file.getName())); + // Save all params. + fileParams.putAll(initMeta.params()); + } + }; + } + }); + + File cacheDirIg0 = cacheWorkDir(snd, DEFAULT_CACHE_NAME); + + File[] cacheParts = cacheDirIg0.listFiles(fileBinFilter); + + for (File file : cacheParts) { + fileSizes.put(file.getName(), file.length()); + fileCrcs.put(file.getName(), FastCrc.calcCrc(file)); + } + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + // Iterate over cache partition cacheParts. + for (File file : cacheParts) { + Map params = new HashMap<>(); + + params.put(file.getName(), file.hashCode()); + + sender.send(file, + // Put additional params to map. + params, + TransmissionPolicy.FILE); + } + } + + log.info("Writing test cacheParts finished. All Ignite instances will be stopped."); + + stopAllGrids(); + + assertEquals(fileSizes.size(), tempStore.listFiles(fileBinFilter).length); + + for (File file : cacheParts) { + // Check received file lenghs + assertEquals("Received file lenght is incorrect: " + file.getName(), + fileSizes.get(file.getName()), new Long(file.length())); + + // Check received params + assertEquals("File additional parameters are not fully transmitted", + fileParams.get(file.getName()), file.hashCode()); + } + + // Check received file CRCs. + for (File file : tempStore.listFiles(fileBinFilter)) { + assertEquals("Received file CRC-32 checksum is incorrect: " + file.getName(), + fileCrcs.get(file.getName()), new Integer(FastCrc.calcCrc(file))); + } + } + + /** + * @throws Exception If fails. + */ + @Test(expected = IgniteCheckedException.class) + public void testFileHandlerOnBeginFails() throws Exception { + final String exTestMessage = "Test exception. Handler initialization failed at onBegin."; + + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData("1Mb", 1024 * 1024); + + rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore) { + @Override public void onBegin(UUID nodeId) { + throw new IgniteException(exTestMessage); + } + + @Override public void onException(UUID nodeId, Throwable err) { + assertEquals(exTestMessage, err.getMessage()); + } + }); + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + } + + /** + * @throws Exception If fails. + */ + @Test(expected = IgniteCheckedException.class) + public void testFileHandlerOnReceiverLeft() throws Exception { + final int fileSizeBytes = 5 * 1024 * 1024; + final AtomicInteger chunksCnt = new AtomicInteger(); + + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData("testFile", fileSizeBytes); + + snd.context().io().transfererFileIoFactory(new FileIOFactory() { + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO fileIo = IO_FACTORY.create(file, modes); + + // Blocking writer and stopping node FileIo. + return new FileIODecorator(fileIo) { + /** {@inheritDoc} */ + @Override public long transferTo(long position, long count, WritableByteChannel target) + throws IOException { + // Send 5 chunks than stop the rcv. + if (chunksCnt.incrementAndGet() == 5) + stopGrid(rcv.name(), true); + + return super.transferTo(position, count, target); + } + }; + } + }); + + rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore)); + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + } + + /** + * @throws Exception If fails. + */ + @Test + public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { + final CountDownLatch latch = new CountDownLatch(1); + + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData("tempFile15Mb", 15 * 1024 * 1024); + + snd.context().io().transfererFileIoFactory(new FileIOFactory() { + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO fileIo = IO_FACTORY.create(file, modes); + + return new FileIODecorator(fileIo) { + /** {@inheritDoc} */ + @Override public long transferTo(long position, long count, WritableByteChannel target) + throws IOException { + + long transferred = super.transferTo(position, count, target); + + stopGrid(snd.name(), true); + + return transferred; + } + }; + } + }); + + rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore)); + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + catch (IgniteCheckedException e) { + // Ignore node stopping exception. + U.log(log,"Expected node stopping exception", e); + } + + assertEquals("Uncomplete resources must be cleaned up on sender left", + 1, // only fileToSend is expected to exist + fileCount(tempStore.toPath())); + } + + /** + * @throws Exception If fails. + */ + @Test(expected = IgniteCheckedException.class) + public void testFileHandlerReconnectOnReadFail() throws Exception { + final String chunkDownloadExMsg = "Test exception. Chunk processing error."; + + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData("testFile", 5 * 1024 * 1024); + final AtomicInteger readedChunks = new AtomicInteger(); + + rcv.context().io().transfererFileIoFactory(new FileIOFactory() { + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + fileIo[0] = IO_FACTORY.create(file, modes); + + // Blocking writer and stopping node FileIo. + return new FileIODecorator(fileIo[0]) { + @Override public long transferFrom(ReadableByteChannel src, long position, long count) + throws IOException { + // Read 4 chunks than throw an exception to emulate error processing. + if (readedChunks.incrementAndGet() == 4) + throw new IgniteException(chunkDownloadExMsg); + + return super.transferFrom(src, position, count); + } + }; + } + }); + + rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore) { + @Override public void onException(UUID nodeId, Throwable err) { + assertEquals(chunkDownloadExMsg, err.getMessage()); + } + }); + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + } + + /** + * @throws Exception If fails. + */ + @Test(expected = IgniteCheckedException.class) + public void testFileHandlerReconnectOnInitFail() throws Exception { + final int fileSizeBytes = 5 * 1024 * 1024; + final AtomicBoolean throwFirstTime = new AtomicBoolean(); + + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData("testFile", fileSizeBytes); + File rcvFile = new File(tempStore, "testFile" + "_" + rcv.localNode().id()); + + rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore) { + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { + if (throwFirstTime.compareAndSet(false, true)) + throw new IgniteException("Test exception. Initialization fail."); + + return rcvFile.getAbsolutePath(); + } + }); + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + + assertEquals(fileToSend.length(), rcvFile.length()); + assertCrcEquals(fileToSend, rcvFile); + } + + /** + * @throws Exception If fails. + */ + @Test + public void testFileHandlerNextWriterOpened() throws Exception { + final int fileSizeBytes = 5 * 1024 * 1024; + final AtomicBoolean networkExThrown = new AtomicBoolean(); + + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData("File_5MB", fileSizeBytes); + File rcvFile = new File(tempStore, "File_5MB" + "_" + rcv.localNode().id()); + + rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore) { + @Override public void onException(UUID nodeId, Throwable err) { + assertEquals("Previous session is not closed properly", IgniteCheckedException.class, err.getClass()); + assertTrue(err.getMessage().startsWith("The handler has been aborted")); + } + + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { + if (networkExThrown.compareAndSet(false, true)) + return null; + + return rcvFile.getAbsolutePath(); + } + }); + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + catch (IgniteCheckedException e) { + // Expected exception. + assertTrue(e.toString(), e.getCause().getMessage().startsWith("Channel processing error")); + } + + //Open next session and complete successfull. + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + + assertEquals(fileToSend.length(), rcvFile.length()); + assertCrcEquals(fileToSend, rcvFile); + + // Remove topic handler and fail + rcv.context().io().removeTransmissionHandler(topic); + + IgniteCheckedException err = null; + + // Open next writer on removed topic. + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + catch (IgniteCheckedException e) { + // Must catch execption here. + err = e; + } + + assertNotNull(err); + } + + /** + * @throws Exception If fails. + */ + @Test(expected = IgniteCheckedException.class) + public void testFileHandlerChannelCloseIfAnotherOpened() throws Exception { + final int fileSizeBytes = 5 * 1024 * 1024; + final CountDownLatch waitLatch = new CountDownLatch(2); + final CountDownLatch completionWait = new CountDownLatch(2); + + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData("file5MBSize", fileSizeBytes); + + rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore) { + @Override public void onBegin(UUID nodeId) { + waitLatch.countDown(); + + try { + waitLatch.await(5, TimeUnit.SECONDS); + } + catch (InterruptedException e) { + throw new IgniteException(e); + } + } + }); + + IgniteCheckedException[] errs = new IgniteCheckedException[1]; + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic); + GridIoManager.TransmissionSender anotherSender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + // Will connect on write attempt. + GridTestUtils.runAsync(() -> { + try { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + catch (IgniteCheckedException e) { + errs[0] = e; + } + finally { + completionWait.countDown(); + } + }); + + GridTestUtils.runAsync(() -> { + try { + anotherSender.send(fileToSend, TransmissionPolicy.FILE); + } + catch (IgniteCheckedException e) { + errs[0] = e; + } + finally { + completionWait.countDown(); + } + }); + + waitLatch.await(5, TimeUnit.SECONDS); + + // Expected that one of the writers will throw exception. + assertFalse("An error must be thrown if connected to the same topic during processing", + errs[0] == null); + + completionWait.await(5, TimeUnit.SECONDS); + + throw errs[0]; + } + } + + /** + * @throws Exception If fails. + */ + @Test + public void testChunkHandlerWithReconnect() throws Exception { + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + final String filePrefix = "testFile"; + final AtomicInteger cnt = new AtomicInteger(); + final AtomicInteger acceptedChunks = new AtomicInteger(); + final File file = new File(tempStore, filePrefix + "_" + rcv.localNode().id()); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData(filePrefix, 10 * 1024 * 1024); + + snd.context().io().transfererFileIoFactory(new FileIOFactory() { + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO fileIo = IO_FACTORY.create(file, modes); + + return new FileIODecorator(fileIo) { + /** {@inheritDoc} */ + @Override public long transferTo(long position, long count, WritableByteChannel target) + throws IOException { + // Send 5 chunks and close the channel. + if (cnt.incrementAndGet() == 10) + target.close(); + + return super.transferTo(position, count, target); + } + }; + } + }); + + rcv.context().io().addTransmissionHandler(topic, new TransmissionHandlerAdapter() { + /** {@inheritDoc} */ + @Override public void onException(UUID nodeId, Throwable err) { + U.closeQuiet(fileIo[0]); + + fileIo[0] = null; + } + + /** {@inheritDoc} */ + @Override public IgniteThrowableConsumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) + throws IgniteCheckedException { + + if (fileIo[0] == null) { + try { + fileIo[0] = IO_FACTORY.create(file); + fileIo[0].position(initMeta.offset()); + } + catch (IOException e) { + throw new IgniteCheckedException(e); + } + } + + return new IgniteThrowableConsumer() { + final LongAdder transferred = new LongAdder(); + + @Override public void accept(ByteBuffer buff) throws IgniteCheckedException { + try { + assertTrue(buff.order() == ByteOrder.nativeOrder()); + assertEquals(0, buff.position()); + assertEquals(buff.limit(), buff.capacity()); + + fileIo[0].writeFully(buff); + + acceptedChunks.getAndIncrement(); + transferred.add(buff.capacity()); + } + catch (Throwable e) { + throw new IgniteCheckedException(e); + } + finally { + closeIfTransferred(); + } + } + + private void closeIfTransferred() { + if (transferred.longValue() == initMeta.count()) { + U.closeQuiet(fileIo[0]); + + fileIo[0] = null; + } + } + }; + } + }); + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.CHUNK); + } + + assertEquals("Total number of accepted chunks by remote node is not as expected", + fileToSend.length() / rcv.configuration().getDataStorageConfiguration().getPageSize(), + acceptedChunks.get()); + assertEquals("Received file and sent files have not the same lenght", fileToSend.length(), file.length()); + assertCrcEquals(fileToSend, file); + assertNull(fileIo[0]); + } + + /** + * @throws Exception If fails. + */ + @Test(expected = IgniteCheckedException.class) + public void testChunkHandlerInitSizeFail() throws Exception { + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData("testFile", 1024 * 1024); + + rcv.context().io().addTransmissionHandler(topic, new TransmissionHandlerAdapter() { + /** {@inheritDoc} */ + @Override public IgniteThrowableConsumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) { + throw new IgniteException("Test exception. Initialization failed"); + } + }); + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.CHUNK); + } + } + + /** + * @param ignite Ignite instance. + * @param cacheName Cache name to add data to. + */ + private void addCacheData(Ignite ignite, String cacheName) { + try (IgniteDataStreamer dataStreamer = ignite.dataStreamer(cacheName)) { + dataStreamer.allowOverwrite(true); + + for (int i = 0; i < CACHE_SIZE; i++) { + if ((i + 1) % (CACHE_SIZE / 10) == 0) + log.info("Prepared " + (i + 1) * 100 / (CACHE_SIZE) + "% entries."); + + dataStreamer.addData(i, i + cacheName.hashCode()); + } + } + } + + /** + * @param ignite The ignite instance. + * @param cacheName Cache name string representation. + * @return The cache working directory. + */ + private File cacheWorkDir(IgniteEx ignite, String cacheName) { + // Resolve cache directory + IgniteInternalCache cache = ignite.cachex(cacheName); + + FilePageStoreManager pageStoreMgr = (FilePageStoreManager)cache.context() + .shared() + .pageStore(); + + return pageStoreMgr.cacheWorkDir(cache.configuration()); + } + + /** + * @param name The file name to create. + * @param size The file size. + * @throws IOException If fails. + */ + private File createFileRandomData(String name, final int size) throws IOException { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + File out = new File(tempStore, name); + + try (RandomAccessFile raf = new RandomAccessFile(out, "rw")) { + byte[] buf = new byte[size]; + rnd.nextBytes(buf); + raf.write(buf); + } + + return out; + } + + /** + * @param fileToSend Source file to check CRC. + * @param fileReceived Destination file to check CRC. + */ + private static void assertCrcEquals(File fileToSend, File fileReceived) { + try { + assertEquals(FastCrc.calcCrc(fileToSend), FastCrc.calcCrc(fileReceived)); + } + catch (IOException e) { + throw new AssertionError(e); + } + } + + /** + * The defailt implementation of transmit session. + */ + private static class DefaultTransmissionHandler extends TransmissionHandlerAdapter { + /** Ignite recevier node. */ + private final IgniteEx rcv; + + /** File to be send. */ + private final File fileToSend; + + /** Temporary local storage. */ + private final File tempStorage; + + /** + * @param rcv Ignite recevier node. + * @param fileToSend File to be send. + * @param tempStorage Temporary local storage. + */ + public DefaultTransmissionHandler(IgniteEx rcv, File fileToSend, File tempStorage) { + this.rcv = rcv; + this.fileToSend = fileToSend; + this.tempStorage = tempStorage; + } + + /** {@inheritDoc} */ + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { + return new File(tempStorage, fileMeta.name() + "_" + rcv.localNode().id()).getAbsolutePath(); + } + + /** {@inheritDoc} */ + @Override public IgniteThrowableConsumer fileHandler(UUID nodeId, TransmissionMeta initMeta) + throws IgniteCheckedException { + return new IgniteThrowableConsumer() { + @Override public void accept(File file) { + assertEquals(fileToSend.length(), file.length()); + assertCrcEquals(fileToSend, file); + } + }; + } + } + + /** + * The defailt implementation of transmit session. + */ + private static class TransmissionHandlerAdapter implements TransmissionHandler { + /** {@inheritDoc} */ + @Override public void onBegin(UUID nodeId) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { + return null; + } + + /** {@inheritDoc} */ + @Override public IgniteThrowableConsumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) + throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public IgniteThrowableConsumer fileHandler(UUID nodeId, TransmissionMeta initMeta) + throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public void onEnd(UUID nodeId) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onException(UUID nodeId, Throwable err) { + // No-op. + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index 3d5bfe2c60467..5cfdb51545824 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -45,6 +45,7 @@ import org.apache.ignite.internal.TransactionsMXBeanImplTest; import org.apache.ignite.internal.managers.IgniteDiagnosticMessagesMultipleConnectionsTest; import org.apache.ignite.internal.managers.IgniteDiagnosticMessagesTest; +import org.apache.ignite.internal.managers.communication.GridIoManagerFileTransmissionSelfTest; import org.apache.ignite.internal.managers.discovery.IncompleteDeserializationExceptionTest; import org.apache.ignite.internal.pagemem.wal.record.WALRecordTest; import org.apache.ignite.internal.processors.DeadLockOnNodeLeftExchangeTest; @@ -250,7 +251,9 @@ ClassPathContentLoggingTest.class, - IncompleteDeserializationExceptionTest.class + IncompleteDeserializationExceptionTest.class, + + GridIoManagerFileTransmissionSelfTest.class }) public class IgniteBasicTestSuite { } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java index a4dfa16f81ba3..250fe2d87993e 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java @@ -29,15 +29,10 @@ import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.stream.Collectors; -import java.util.stream.Stream; - import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.cache.query.QueryRetryException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheContextInfo; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; @@ -53,7 +48,6 @@ import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex; import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndexBase; import org.apache.ignite.internal.processors.query.h2.database.IndexInformation; -import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter; @@ -169,10 +163,6 @@ public class GridH2Table extends TableBase { /** Table statistics. */ private volatile TableStatistics tblStats; - /** Logger. */ - @GridToStringExclude - private IgniteLogger log; - /** * Creates table. * @@ -241,12 +231,6 @@ public GridH2Table( // Init stats with the dummy values. This prevents us from scanning index with backup filter when // topology may not be initialized yet. tblStats = new TableStatistics(0, 0); - - if (desc != null && desc.context() != null) { - GridKernalContext ctx = desc.context().kernalContext(); - - log = ctx.log(getClass()); - } } /** @@ -873,52 +857,6 @@ public boolean rebuildFromHashInProgress() { return commitUserIndex(ses, idxName); } - /** - * Checks index presence, return {@link Index} if index with same name or same fields and search direction already - * exist or {@code null} othervise. - * - * @param curIdx Index to check. - * @return Index if equal or subset index exist. - * @throws IgniteCheckedException If failed. - */ - private @Nullable Index checkIndexPresence(Index curIdx) throws IgniteCheckedException { - IndexColumn[] curColumns = curIdx.getIndexColumns(); - - Index registredIdx = null; - - for (Index idx : idxs) { - if (!(idx instanceof H2TreeIndex)) - continue; - - if (F.eq(curIdx.getName(), idx.getName())) - throw new IgniteCheckedException("Index already exists: " + idx.getName()); - - IndexColumn[] idxColumns = idx.getIndexColumns(); - - for (int i = 0; i < Math.min(idxColumns.length, curColumns.length); ++i) { - IndexColumn idxCol = idxColumns[i]; - IndexColumn curCol = curColumns[i]; - - // pk attach at the end of listed fields. - if (curCol.column.getColumnId() == 0 && registredIdx != null) - continue; - - if (H2Utils.equals(idxCol, curCol) && idxCol.sortType == curCol.sortType) - registredIdx = idx; - else { - registredIdx = null; - - break; - } - } - - if (registredIdx != null) - return registredIdx; - } - - return null; - } - /** * Add index that is in an intermediate state and is still being built, thus is not used in queries until it is * promoted. @@ -934,17 +872,9 @@ public void proposeUserIndex(Index idx) throws IgniteCheckedException { try { ensureNotDestroyed(); - Index idxExist = checkIndexPresence(idx); - - if (idxExist != null) { - String idxCols = Stream.of(idxExist.getIndexColumns()) - .map(k -> k.columnName).collect(Collectors.joining(", ")); - - U.warn(log, "Index with the given set or subset of columns already exists " + - "(consider dropping either new or existing index) [cacheName=" + cacheInfo.name() + ", " + - "schemaName=" + getSchema().getName() + ", tableName=" + getName() + - ", newIndexName=" + idx.getName() + ", existingIndexName=" + idxExist.getName() + - ", existingIndexColumns=[" + idxCols + "]]"); + for (Index oldIdx : idxs) { + if (F.eq(oldIdx.getName(), idx.getName())) + throw new IgniteCheckedException("Index already exists: " + idx.getName()); } Index oldTmpIdx = tmpIdxs.put(idx.getName(), (GridH2IndexBase)idx); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java index 6ee3e535ef187..8163a76935556 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java @@ -199,7 +199,7 @@ public void testIndexCreationWhenNodeStopped() throws Exception { " ADDRESS VARCHAR,\n" + " LANG VARCHAR,\n" + " BIRTH_DATE TIMESTAMP,\n" + - " CONSTRAINT PK_PERSON PRIMARY KEY (FIRST_NAME,LAST_NAME,ADDRESS,LANG)\n" + + " CONSTRAINT PK_PESON PRIMARY KEY (FIRST_NAME,LAST_NAME,ADDRESS,LANG)\n" + " ) WITH \"key_type=PersonKeyType, CACHE_NAME=PersonCache, value_type=PersonValueType, AFFINITY_KEY=FIRST_NAME,template=PARTITIONED,backups=1\""); try (PreparedStatement stmt = conn.prepareStatement( diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java index 553d10a86ffe0..dae6dcd438ac3 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java @@ -26,13 +26,10 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Objects; -import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; -import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataRegionConfiguration; @@ -41,34 +38,15 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; -import org.apache.ignite.internal.processors.query.GridQueryProcessor; -import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.testframework.ListeningTestLogger; -import org.apache.ignite.testframework.LogListener; -import org.jetbrains.annotations.Nullable; import org.junit.Test; -import static org.apache.ignite.internal.processors.query.h2.opt.H2TableScanIndex.SCAN_INDEX_NAME_SUFFIX; - /** * A set of basic tests for caches with indexes. */ public class BasicIndexTest extends AbstractIndexingCommonTest { - /** Default client name. */ - private static final String CLIENT_NAME = "client"; - - /** {@code True} If index need to be created throught static config. */ - private static boolean createIdx = true; - - /** {@code True} If cache nedd to be created throught static config. */ - private static boolean createStaticCache = true; - - /** Default table name. */ - private static final String TEST_TBL_NAME = "PUBLIC.TEST_TABLE"; - /** */ private Collection indexes = Collections.emptyList(); @@ -81,12 +59,6 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { /** */ private int gridCount = 1; - /** Server listening logger. */ - private ListeningTestLogger srvLog; - - /** Client listening logger. */ - private ListeningTestLogger clientLog; - /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { assertNotNull(inlineSize); @@ -98,17 +70,6 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { igniteCfg.setConsistentId(igniteInstanceName); - if (igniteInstanceName.startsWith(CLIENT_NAME)) { - igniteCfg.setClientMode(true); - - if (clientLog != null) - igniteCfg.setGridLogger(clientLog); - } - else { - if (srvLog != null) - igniteCfg.setGridLogger(srvLog); - } - LinkedHashMap fields = new LinkedHashMap<>(); fields.put("keyStr", String.class.getName()); fields.put("keyLong", Long.class.getName()); @@ -117,11 +78,7 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { fields.put("valLong", Long.class.getName()); fields.put("valPojo", Pojo.class.getName()); - if (!createIdx) - indexes = Collections.emptyList(); - CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME) - .setAffinity(new RendezvousAffinityFunction(false, 32)) .setQueryEntities(Collections.singleton( new QueryEntity() .setKeyType(Key.class.getName()) @@ -129,17 +86,15 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { .setFields(fields) .setKeyFields(new HashSet<>(Arrays.asList("keyStr", "keyLong", "keyPojo"))) .setIndexes(indexes) - .setAliases(Collections.singletonMap(QueryUtils.KEY_FIELD_NAME, "pk_id")) )) .setSqlIndexMaxInlineSize(inlineSize); - if (createStaticCache) - igniteCfg.setCacheConfiguration(ccfg); + igniteCfg.setCacheConfiguration(ccfg); if (isPersistenceEnabled) { igniteCfg.setDataStorageConfiguration(new DataStorageConfiguration() .setDefaultDataRegionConfiguration( - new DataRegionConfiguration().setPersistenceEnabled(true).setMaxSize(10 * 1024 * 1024) + new DataRegionConfiguration().setPersistenceEnabled(true) ) ); } @@ -166,8 +121,6 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { cleanPersistenceDir(); - srvLog = clientLog = null; - super.afterTest(); } @@ -256,442 +209,6 @@ public void testDynamicIndexesNoPersistence() throws Exception { } } - /** - * Tests mixed dynamic and static caches with indexes creation. - * - * @throws Exception If failed. - */ - @Test - public void testDynamicIdxOnStaticCacheWithIdxWithoutPersistence() throws Exception { - runDynamicIdxOnStaticCacheWithIdx(false); - } - - /** - * Tests mixed dynamic and static caches with indexes creation. - * - * @throws Exception If failed. - */ - @Test - public void testDynamicIdxOnStaticCacheWithIdxWithPersistence() throws Exception { - runDynamicIdxOnStaticCacheWithIdx(true); - } - - /** */ - private void runDynamicIdxOnStaticCacheWithIdx(boolean persistEnabled) throws Exception { - isPersistenceEnabled = persistEnabled; - - inlineSize = 10; - - createIdx = false; - - indexes = Collections.singletonList(new QueryIndex("valStr")); - - IgniteEx ig0 = startGrid(0); - - createIdx = true; - - startGrid(1); - - if (persistEnabled) - ig0.cluster().active(true); - - IgniteCache cache = grid(0).cache(DEFAULT_CACHE_NAME); - - populateCache(); - - String plan = cache.query(new SqlFieldsQuery("explain select * from Val where valStr between 0 and ?") - .setArgs(100)).getAll().get(0).get(0).toString(); - - assertTrue(plan, plan.contains(SCAN_INDEX_NAME_SUFFIX)); - - stopAllGrids(); - - if (persistEnabled) - cleanPersistenceDir(); - - createStaticCache = false; - - ig0 = startGrid(0); - - if (persistEnabled) - ig0.cluster().active(true); - - ig0.getOrCreateCache(DEFAULT_CACHE_NAME); - - populateCache(); - - createStaticCache = true; - - try { - startGrid(1); - - fail("Exception wasn't thrown"); - } - catch (IgniteCheckedException e) { - // no op. - } - } - - /** - * Tests dynamic indexes creation with equal fields. - * - * @throws Exception If failed. - */ - @Test - public void testEqualFieldsDynamicIndexesWithoutPersistence() throws Exception { - runEqualFieldsDynamicIndexes(false); - } - - /** - * Tests dynamic indexes creation with equal fields. - * - * @throws Exception If failed. - */ - @Test - public void testEqualFieldsDynamicIndexesWithPersistence() throws Exception { - runEqualFieldsDynamicIndexes(true); - } - - /** */ - private void runEqualFieldsDynamicIndexes(boolean persistEnabled) throws Exception { - isPersistenceEnabled = persistEnabled; - - indexes = Collections.singletonList(new QueryIndex("valStr")); - - inlineSize = 10; - - srvLog = new ListeningTestLogger(false, log); - - clientLog = new ListeningTestLogger(false, log); - - String msg1 = "Index with the given set or subset of columns already exists"; - - LogListener lsnr = LogListener.matches(msg1).andMatches(Pattern.compile(".*newIndexName=idx[0-9]")).build(); - - LogListener staticCachesLsnr = LogListener.matches(msg1).build(); - - srvLog.registerListener(staticCachesLsnr); - - IgniteEx ig0 = startGrid(0); - - if (persistEnabled) - ig0.cluster().active(true); - - IgniteCache cache = grid(0).cache(DEFAULT_CACHE_NAME); - - populateCache(); - - cache.query(new SqlFieldsQuery("create index \"idx0\" on Val(valStr)")); - - assertTrue(staticCachesLsnr.check()); - - srvLog.unregisterListener(staticCachesLsnr); - - srvLog.registerListener(lsnr); - - cache.query(new SqlFieldsQuery("create index \"idx1\" on Val(valStr, valLong)")); - - cache.query(new SqlFieldsQuery("create index \"idx2\" on Val(valStr desc, valLong)")); - - assertFalse(lsnr.check()); - - cache.query(new SqlFieldsQuery("create index \"idx3\" on Val(valStr, valLong)")); - - cache.query(new SqlFieldsQuery("create index \"idx4\" on Val(valLong)")); - - assertTrue(lsnr.check()); - - srvLog.unregisterListener(lsnr); - - IgniteEx client = startGrid(CLIENT_NAME); - - cache = client.cache(DEFAULT_CACHE_NAME); - - LogListener lsnrIdx5 = LogListener.matches(msg1).andMatches("idx5").build(); - - srvLog.registerListener(lsnrIdx5); - - cache.query(new SqlFieldsQuery("create index \"idx5\" on Val(valStr desc, valLong)")); - - assertTrue(lsnrIdx5.check()); - - LogListener lsnrIdx7 = LogListener.matches(msg1).andMatches("idx7").build(); - - srvLog.registerListener(lsnrIdx7); - - cache.query(new SqlFieldsQuery("create index \"idx6\" on Val(valLong)")); - - cache.query(new SqlFieldsQuery("create index \"idx7\" on Val(keyStr, keyLong, keyPojo, valLong)")); - - assertFalse(lsnrIdx7.check()); - } - - /** */ - private boolean checkIdxUsed(GridQueryProcessor qryProc, @Nullable String idxName, String tblName, String... reqFlds) { - String sql = "explain select * from " + tblName + " where "; - - for (int i = 0; i < reqFlds.length; ++i) - sql += reqFlds[i] + " > 0 " + ((i < reqFlds.length - 1) ? " and " : ""); - - String plan = qryProc.querySqlFields(new SqlFieldsQuery(sql), true) - .getAll().get(0).get(0).toString().toUpperCase(); - - return idxName != null ? (!plan.contains(SCAN_INDEX_NAME_SUFFIX) && plan.contains(idxName.toUpperCase())) : !plan.contains(SCAN_INDEX_NAME_SUFFIX); - } - - /** */ - private boolean checkIdxAlreadyExistLog(GridQueryProcessor qryProc, String idxName, String tblName, String... reqFlds) { - String msg0 = "Index with the given set or subset of columns already exists"; - - String sql = "create index \"" + idxName + "\" on " + tblName + "("; - - for (int i = 0; i < reqFlds.length; ++i) - sql += reqFlds[i] + ((i < reqFlds.length - 1) ? ", " : ")"); - - LogListener lsnrIdx = LogListener.matches(msg0).andMatches(idxName).build(); - - srvLog.registerListener(lsnrIdx); - - qryProc.querySqlFields(new SqlFieldsQuery(sql), true).getAll(); - - return lsnrIdx.check(); - } - - /** */ - private void populateTable(GridQueryProcessor qryProc, String tblName, int consPkFldsNum, String... reqFlds) { - assert consPkFldsNum <= reqFlds.length; - - String sql = "CREATE TABLE " + tblName + " ("; - - String sqlIns = "INSERT INTO " + tblName + " ("; - - for (int i = 0; i < reqFlds.length; ++i) { - sql += reqFlds[i] + " VARCHAR, "; - - sqlIns += reqFlds[i] + ((i < reqFlds.length - 1) ? ", " : ") values ("); - } - - if (consPkFldsNum > 0) { - sql += " CONSTRAINT PK_PERSON PRIMARY KEY ("; - - for (int i = 0; i < consPkFldsNum; ++i) - sql += reqFlds[i] + ((i < consPkFldsNum - 1) ? ", " : "))"); - } - else - sql += ")"; - - qryProc.querySqlFields(new SqlFieldsQuery(sql), true); - - for (int i = 0; i < 10; ++i) { - String s0 = sqlIns; - - for (int f = 0; f < reqFlds.length; ++f) - s0 += i + ((f < reqFlds.length - 1) ? ", " : ")"); - - qryProc.querySqlFields(new SqlFieldsQuery(s0), true).getAll(); - } - } - - /** - * Checks index usage for full coverage. - */ - @Test - public void testAllTableFieldsCoveredByIdx() throws Exception { - inlineSize = 10; - - srvLog = new ListeningTestLogger(false, log); - - IgniteEx ig0 = startGrid(0); - - GridQueryProcessor qryProc = ig0.context().query(); - - populateTable(qryProc, TEST_TBL_NAME, 2, "FIRST_NAME", "LAST_NAME", - "ADDRESS", "LANG"); - - assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LANG")); - - assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LAST_NAME")); - - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME")); - - assertTrue(checkIdxUsed(qryProc, "_key_PK", TEST_TBL_NAME, "FIRST_NAME", - "LAST_NAME", "LANG", "ADDRESS")); - - assertTrue(checkIdxAlreadyExistLog( - qryProc, "idx1", TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME")); - - String sqlIdx2 = String.format("create index \"idx2\" on %s(LANG, ADDRESS)", TEST_TBL_NAME); - - qryProc.querySqlFields(new SqlFieldsQuery(sqlIdx2), true).getAll(); - - assertTrue(checkIdxUsed(qryProc, "idx2", TEST_TBL_NAME, "FIRST_NAME", - "LAST_NAME", "LANG", "ADDRESS")); - - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", - "LAST_NAME", "ADDRESS", "LANG")); - // first idx fields not belongs to request fields. - assertTrue(checkIdxUsed(qryProc, "idx2", TEST_TBL_NAME, "ADDRESS", - "LAST_NAME")); - } - - /** - * Checks index usage for partial coverage. - * Last field not participate in any index. - */ - @Test - public void testPartialTableFieldsCoveredByIdx() throws Exception { - inlineSize = 10; - - String msg0 = "Index with the given set or subset of columns already exists"; - - srvLog = new ListeningTestLogger(false, log); - - IgniteEx ig0 = startGrid(0); - - GridQueryProcessor qryProc = ig0.context().query(); - - String cacheName = QueryUtils.createTableCacheName("PUBLIC", "TEST_TABLE"); - - populateTable(qryProc, TEST_TBL_NAME, 2, "FIRST_NAME", "LAST_NAME", - "ADDRESS", "LANG", "GENDER"); - - IgniteCache jcache = ig0.cache(cacheName); - - assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LANG")); - - assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LAST_NAME")); - - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME")); - - assertTrue(checkIdxUsed(qryProc, "_key_PK", TEST_TBL_NAME, "FIRST_NAME", - "LAST_NAME", "LANG", "ADDRESS")); - - assertTrue(checkIdxAlreadyExistLog( - qryProc, "idx1", TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME")); - - String sqlIdx2 = String.format("create index \"idx2\" on %s(LANG, ADDRESS)", TEST_TBL_NAME); - - qryProc.querySqlFields(new SqlFieldsQuery(sqlIdx2), true).getAll(); - // _key_PK used. - assertFalse(checkIdxUsed(qryProc, "idx2", TEST_TBL_NAME, "FIRST_NAME", - "LAST_NAME", "LANG", "ADDRESS")); - // _key_PK used. - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", - "LAST_NAME", "LANG", "ADDRESS")); - - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", - "LAST_NAME", "ADDRESS", "LANG")); - // first idx fields not belongs to request fields. - assertFalse(checkIdxUsed(qryProc, "idx2", TEST_TBL_NAME, "ADDRESS", - "LAST_NAME")); - - assertFalse(checkIdxAlreadyExistLog( - qryProc, "idx3", TEST_TBL_NAME, "ADDRESS", "LANG")); - - assertTrue(checkIdxAlreadyExistLog( - qryProc, "idx4", TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "ADDRESS", "LANG")); - - LogListener lsnrIdx4 = LogListener.matches(msg0).andMatches("_key_PK").build(); - - srvLog.registerListener(lsnrIdx4); - - String sqlIdx5 = String.format("create index \"idx5\" on %s(FIRST_NAME, LAST_NAME, LANG, ADDRESS)", TEST_TBL_NAME); - - jcache.query(new SqlFieldsQuery(sqlIdx5)).getAll(); - - assertTrue(lsnrIdx4.check()); - } - - /** - * Tests different fields sequence in indexes. - */ - @Test - public void testCreateIdxWithDifferentIdxFldsSeq() throws Exception { - inlineSize = 10; - - srvLog = new ListeningTestLogger(false, log); - - IgniteEx ig0 = startGrid(0); - - IgniteEx client = startGrid(CLIENT_NAME); - - GridQueryProcessor qryProc = ig0.context().query(); - - populateTable(qryProc, TEST_TBL_NAME, 1, "c1", "c2", "c3", "c4", "c5"); - - assertFalse(checkIdxAlreadyExistLog( - qryProc, "idx1", TEST_TBL_NAME, "c1", "c2", "c3", "c4", "c5")); - - assertFalse(checkIdxAlreadyExistLog( - qryProc, "idx2", TEST_TBL_NAME, "c1", "c3", "c4", "c5")); - - assertTrue(checkIdxAlreadyExistLog( - qryProc, "idx3", TEST_TBL_NAME, "c1", "c2")); - - assertTrue(checkIdxAlreadyExistLog( - qryProc, "idx4", TEST_TBL_NAME, "c1", "c3")); - - assertFalse(checkIdxAlreadyExistLog( - qryProc, "idx5", TEST_TBL_NAME, "c1", "c4", "c5")); - - GridQueryProcessor qryProcCl = client.context().query(); - - assertTrue(checkIdxAlreadyExistLog( - qryProcCl, "idx6", TEST_TBL_NAME, "c1", "c2")); - } - - /** - * Tests different fields sequence in indexes. - * Last field not participate in any index. - */ - @Test - public void testIndexWithDifferentFldsReqPartialFldsInIdx() throws Exception { - inlineSize = 10; - - IgniteEx ig0 = startGrid(0); - - GridQueryProcessor qryProc = ig0.context().query(); - - populateTable(qryProc, TEST_TBL_NAME, 2, "FIRST_NAME", "LAST_NAME", - "ADDRESS", "LANG", "GENDER"); - - String sqlIdx1 = String.format("create index \"idx1\" on %s(LANG, LAST_NAME, ADDRESS, FIRST_NAME)", TEST_TBL_NAME); - - qryProc.querySqlFields(new SqlFieldsQuery(sqlIdx1), true).getAll(); - - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "LANG")); - - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "ADDRESS")); - - assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LAST_NAME", "ADDRESS")); - } - - /** - * Tests different fields sequence in indexes. - * All fields covered by indexes. - */ - @Test - public void testIndexWithDifferentFldsReqAllFldsInIdx() throws Exception { - inlineSize = 10; - - IgniteEx ig0 = startGrid(0); - - GridQueryProcessor qryProc = ig0.context().query(); - - populateTable(qryProc, TEST_TBL_NAME, 2, "FIRST_NAME", "LAST_NAME", - "ADDRESS", "LANG"); - - String sqlIdx1 = String.format("create index \"idx1\" on %s(LANG, LAST_NAME, ADDRESS, FIRST_NAME)", TEST_TBL_NAME); - - qryProc.querySqlFields(new SqlFieldsQuery(sqlIdx1), true).getAll(); - - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "LANG")); - - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "ADDRESS")); - - assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LAST_NAME", "ADDRESS")); - } - /** */ @Test public void testNoIndexesWithPersistence() throws Exception { From 2f26491cf57000711c77d1978057f044a004c16d Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 19:15:05 +0300 Subject: [PATCH 02/69] IGNITE-10619: move session methods to impl --- .../ignite/internal/util/nio/GridNioSession.java | 14 -------------- .../internal/util/nio/GridNioSessionImpl.java | 12 ++++++++---- 2 files changed, 8 insertions(+), 18 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java index 8d4af8bcb1772..12b9b40b10c34 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java @@ -70,20 +70,6 @@ public interface GridNioSession { */ public long closeTime(); - /** - * @return {@code True} to close SocketChannel on current session close occured. - */ - public default boolean closeSocketOnSessionClose() { - return true; - } - - /** - * @param closeSocket {@code False} remain SocketChannel open on session close. - */ - public default void closeSocketOnSessionClose(boolean closeSocket) { - throw new UnsupportedOperationException(); - } - /** * Returns the time when last read activity was performed on this session. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java index ce951b18bfe9a..89ece47017c70 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java @@ -327,13 +327,17 @@ public boolean setClosed() { return closeTime.compareAndSet(0, U.currentTimeMillis()); } - /** {@inheritDoc} */ - @Override public boolean closeSocketOnSessionClose() { + /** + * @return {@code True} to close SocketChannel on current session close occured. + */ + public boolean closeSocketOnSessionClose() { return closeSocket; } - /** {@inheritDoc} */ - @Override public void closeSocketOnSessionClose(boolean closeSocket) { + /** + * @param closeSocket {@code False} remain SocketChannel open on session close. + */ + public void closeSocketOnSessionClose(boolean closeSocket) { this.closeSocket = closeSocket; } From b2008f304e401fbffe2bab99ba8f8929ffb1a90e Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 19:17:49 +0300 Subject: [PATCH 03/69] IGNITE-10619: revert changes from master --- .../processors/query/h2/opt/GridH2Table.java | 75 ++- .../cache/IgniteDynamicSqlRestoreTest.java | 2 +- .../cache/index/BasicIndexTest.java | 487 +++++++++++++++++- 3 files changed, 558 insertions(+), 6 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java index 250fe2d87993e..666d712190ecf 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java @@ -29,9 +29,13 @@ import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteInterruptedException; +import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.query.QueryRetryException; +import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheContextInfo; @@ -48,6 +52,7 @@ import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex; import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndexBase; import org.apache.ignite.internal.processors.query.h2.database.IndexInformation; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter; @@ -163,6 +168,10 @@ public class GridH2Table extends TableBase { /** Table statistics. */ private volatile TableStatistics tblStats; + /** Logger. */ + @GridToStringExclude + private IgniteLogger log; + /** * Creates table. * @@ -231,6 +240,12 @@ public GridH2Table( // Init stats with the dummy values. This prevents us from scanning index with backup filter when // topology may not be initialized yet. tblStats = new TableStatistics(0, 0); + + if (desc != null && desc.context() != null) { + GridKernalContext ctx = desc.context().kernalContext(); + + log = ctx.log(getClass()); + } } /** @@ -857,6 +872,52 @@ public boolean rebuildFromHashInProgress() { return commitUserIndex(ses, idxName); } + /** + * Checks index presence, return {@link Index} if index with same name or same fields and search direction already + * exist or {@code null} othervise. + * + * @param curIdx Index to check. + * @return Index if equal or subset index exist. + * @throws IgniteCheckedException If failed. + */ + private @Nullable Index checkIndexPresence(Index curIdx) throws IgniteCheckedException { + IndexColumn[] curColumns = curIdx.getIndexColumns(); + + Index registredIdx = null; + + for (Index idx : idxs) { + if (!(idx instanceof H2TreeIndex)) + continue; + + if (F.eq(curIdx.getName(), idx.getName())) + throw new IgniteCheckedException("Index already exists: " + idx.getName()); + + IndexColumn[] idxColumns = idx.getIndexColumns(); + + for (int i = 0; i < Math.min(idxColumns.length, curColumns.length); ++i) { + IndexColumn idxCol = idxColumns[i]; + IndexColumn curCol = curColumns[i]; + + // pk attach at the end of listed fields. + if (curCol.column.getColumnId() == 0 && registredIdx != null) + continue; + + if (H2Utils.equals(idxCol, curCol) && idxCol.sortType == curCol.sortType) + registredIdx = idx; + else { + registredIdx = null; + + break; + } + } + + if (registredIdx != null) + return registredIdx; + } + + return null; + } + /** * Add index that is in an intermediate state and is still being built, thus is not used in queries until it is * promoted. @@ -872,9 +933,17 @@ public void proposeUserIndex(Index idx) throws IgniteCheckedException { try { ensureNotDestroyed(); - for (Index oldIdx : idxs) { - if (F.eq(oldIdx.getName(), idx.getName())) - throw new IgniteCheckedException("Index already exists: " + idx.getName()); + Index idxExist = checkIndexPresence(idx); + + if (idxExist != null) { + String idxCols = Stream.of(idxExist.getIndexColumns()) + .map(k -> k.columnName).collect(Collectors.joining(", ")); + + U.warn(log, "Index with the given set or subset of columns already exists " + + "(consider dropping either new or existing index) [cacheName=" + cacheInfo.name() + ", " + + "schemaName=" + getSchema().getName() + ", tableName=" + getName() + + ", newIndexName=" + idx.getName() + ", existingIndexName=" + idxExist.getName() + + ", existingIndexColumns=[" + idxCols + "]]"); } Index oldTmpIdx = tmpIdxs.put(idx.getName(), (GridH2IndexBase)idx); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java index 8163a76935556..6ee3e535ef187 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java @@ -199,7 +199,7 @@ public void testIndexCreationWhenNodeStopped() throws Exception { " ADDRESS VARCHAR,\n" + " LANG VARCHAR,\n" + " BIRTH_DATE TIMESTAMP,\n" + - " CONSTRAINT PK_PESON PRIMARY KEY (FIRST_NAME,LAST_NAME,ADDRESS,LANG)\n" + + " CONSTRAINT PK_PERSON PRIMARY KEY (FIRST_NAME,LAST_NAME,ADDRESS,LANG)\n" + " ) WITH \"key_type=PersonKeyType, CACHE_NAME=PersonCache, value_type=PersonValueType, AFFINITY_KEY=FIRST_NAME,template=PARTITIONED,backups=1\""); try (PreparedStatement stmt = conn.prepareStatement( diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java index dae6dcd438ac3..553d10a86ffe0 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java @@ -26,10 +26,13 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Objects; +import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataRegionConfiguration; @@ -38,15 +41,34 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.ListeningTestLogger; +import org.apache.ignite.testframework.LogListener; +import org.jetbrains.annotations.Nullable; import org.junit.Test; +import static org.apache.ignite.internal.processors.query.h2.opt.H2TableScanIndex.SCAN_INDEX_NAME_SUFFIX; + /** * A set of basic tests for caches with indexes. */ public class BasicIndexTest extends AbstractIndexingCommonTest { + /** Default client name. */ + private static final String CLIENT_NAME = "client"; + + /** {@code True} If index need to be created throught static config. */ + private static boolean createIdx = true; + + /** {@code True} If cache nedd to be created throught static config. */ + private static boolean createStaticCache = true; + + /** Default table name. */ + private static final String TEST_TBL_NAME = "PUBLIC.TEST_TABLE"; + /** */ private Collection indexes = Collections.emptyList(); @@ -59,6 +81,12 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { /** */ private int gridCount = 1; + /** Server listening logger. */ + private ListeningTestLogger srvLog; + + /** Client listening logger. */ + private ListeningTestLogger clientLog; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { assertNotNull(inlineSize); @@ -70,6 +98,17 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { igniteCfg.setConsistentId(igniteInstanceName); + if (igniteInstanceName.startsWith(CLIENT_NAME)) { + igniteCfg.setClientMode(true); + + if (clientLog != null) + igniteCfg.setGridLogger(clientLog); + } + else { + if (srvLog != null) + igniteCfg.setGridLogger(srvLog); + } + LinkedHashMap fields = new LinkedHashMap<>(); fields.put("keyStr", String.class.getName()); fields.put("keyLong", Long.class.getName()); @@ -78,7 +117,11 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { fields.put("valLong", Long.class.getName()); fields.put("valPojo", Pojo.class.getName()); + if (!createIdx) + indexes = Collections.emptyList(); + CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME) + .setAffinity(new RendezvousAffinityFunction(false, 32)) .setQueryEntities(Collections.singleton( new QueryEntity() .setKeyType(Key.class.getName()) @@ -86,15 +129,17 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { .setFields(fields) .setKeyFields(new HashSet<>(Arrays.asList("keyStr", "keyLong", "keyPojo"))) .setIndexes(indexes) + .setAliases(Collections.singletonMap(QueryUtils.KEY_FIELD_NAME, "pk_id")) )) .setSqlIndexMaxInlineSize(inlineSize); - igniteCfg.setCacheConfiguration(ccfg); + if (createStaticCache) + igniteCfg.setCacheConfiguration(ccfg); if (isPersistenceEnabled) { igniteCfg.setDataStorageConfiguration(new DataStorageConfiguration() .setDefaultDataRegionConfiguration( - new DataRegionConfiguration().setPersistenceEnabled(true) + new DataRegionConfiguration().setPersistenceEnabled(true).setMaxSize(10 * 1024 * 1024) ) ); } @@ -121,6 +166,8 @@ public class BasicIndexTest extends AbstractIndexingCommonTest { cleanPersistenceDir(); + srvLog = clientLog = null; + super.afterTest(); } @@ -209,6 +256,442 @@ public void testDynamicIndexesNoPersistence() throws Exception { } } + /** + * Tests mixed dynamic and static caches with indexes creation. + * + * @throws Exception If failed. + */ + @Test + public void testDynamicIdxOnStaticCacheWithIdxWithoutPersistence() throws Exception { + runDynamicIdxOnStaticCacheWithIdx(false); + } + + /** + * Tests mixed dynamic and static caches with indexes creation. + * + * @throws Exception If failed. + */ + @Test + public void testDynamicIdxOnStaticCacheWithIdxWithPersistence() throws Exception { + runDynamicIdxOnStaticCacheWithIdx(true); + } + + /** */ + private void runDynamicIdxOnStaticCacheWithIdx(boolean persistEnabled) throws Exception { + isPersistenceEnabled = persistEnabled; + + inlineSize = 10; + + createIdx = false; + + indexes = Collections.singletonList(new QueryIndex("valStr")); + + IgniteEx ig0 = startGrid(0); + + createIdx = true; + + startGrid(1); + + if (persistEnabled) + ig0.cluster().active(true); + + IgniteCache cache = grid(0).cache(DEFAULT_CACHE_NAME); + + populateCache(); + + String plan = cache.query(new SqlFieldsQuery("explain select * from Val where valStr between 0 and ?") + .setArgs(100)).getAll().get(0).get(0).toString(); + + assertTrue(plan, plan.contains(SCAN_INDEX_NAME_SUFFIX)); + + stopAllGrids(); + + if (persistEnabled) + cleanPersistenceDir(); + + createStaticCache = false; + + ig0 = startGrid(0); + + if (persistEnabled) + ig0.cluster().active(true); + + ig0.getOrCreateCache(DEFAULT_CACHE_NAME); + + populateCache(); + + createStaticCache = true; + + try { + startGrid(1); + + fail("Exception wasn't thrown"); + } + catch (IgniteCheckedException e) { + // no op. + } + } + + /** + * Tests dynamic indexes creation with equal fields. + * + * @throws Exception If failed. + */ + @Test + public void testEqualFieldsDynamicIndexesWithoutPersistence() throws Exception { + runEqualFieldsDynamicIndexes(false); + } + + /** + * Tests dynamic indexes creation with equal fields. + * + * @throws Exception If failed. + */ + @Test + public void testEqualFieldsDynamicIndexesWithPersistence() throws Exception { + runEqualFieldsDynamicIndexes(true); + } + + /** */ + private void runEqualFieldsDynamicIndexes(boolean persistEnabled) throws Exception { + isPersistenceEnabled = persistEnabled; + + indexes = Collections.singletonList(new QueryIndex("valStr")); + + inlineSize = 10; + + srvLog = new ListeningTestLogger(false, log); + + clientLog = new ListeningTestLogger(false, log); + + String msg1 = "Index with the given set or subset of columns already exists"; + + LogListener lsnr = LogListener.matches(msg1).andMatches(Pattern.compile(".*newIndexName=idx[0-9]")).build(); + + LogListener staticCachesLsnr = LogListener.matches(msg1).build(); + + srvLog.registerListener(staticCachesLsnr); + + IgniteEx ig0 = startGrid(0); + + if (persistEnabled) + ig0.cluster().active(true); + + IgniteCache cache = grid(0).cache(DEFAULT_CACHE_NAME); + + populateCache(); + + cache.query(new SqlFieldsQuery("create index \"idx0\" on Val(valStr)")); + + assertTrue(staticCachesLsnr.check()); + + srvLog.unregisterListener(staticCachesLsnr); + + srvLog.registerListener(lsnr); + + cache.query(new SqlFieldsQuery("create index \"idx1\" on Val(valStr, valLong)")); + + cache.query(new SqlFieldsQuery("create index \"idx2\" on Val(valStr desc, valLong)")); + + assertFalse(lsnr.check()); + + cache.query(new SqlFieldsQuery("create index \"idx3\" on Val(valStr, valLong)")); + + cache.query(new SqlFieldsQuery("create index \"idx4\" on Val(valLong)")); + + assertTrue(lsnr.check()); + + srvLog.unregisterListener(lsnr); + + IgniteEx client = startGrid(CLIENT_NAME); + + cache = client.cache(DEFAULT_CACHE_NAME); + + LogListener lsnrIdx5 = LogListener.matches(msg1).andMatches("idx5").build(); + + srvLog.registerListener(lsnrIdx5); + + cache.query(new SqlFieldsQuery("create index \"idx5\" on Val(valStr desc, valLong)")); + + assertTrue(lsnrIdx5.check()); + + LogListener lsnrIdx7 = LogListener.matches(msg1).andMatches("idx7").build(); + + srvLog.registerListener(lsnrIdx7); + + cache.query(new SqlFieldsQuery("create index \"idx6\" on Val(valLong)")); + + cache.query(new SqlFieldsQuery("create index \"idx7\" on Val(keyStr, keyLong, keyPojo, valLong)")); + + assertFalse(lsnrIdx7.check()); + } + + /** */ + private boolean checkIdxUsed(GridQueryProcessor qryProc, @Nullable String idxName, String tblName, String... reqFlds) { + String sql = "explain select * from " + tblName + " where "; + + for (int i = 0; i < reqFlds.length; ++i) + sql += reqFlds[i] + " > 0 " + ((i < reqFlds.length - 1) ? " and " : ""); + + String plan = qryProc.querySqlFields(new SqlFieldsQuery(sql), true) + .getAll().get(0).get(0).toString().toUpperCase(); + + return idxName != null ? (!plan.contains(SCAN_INDEX_NAME_SUFFIX) && plan.contains(idxName.toUpperCase())) : !plan.contains(SCAN_INDEX_NAME_SUFFIX); + } + + /** */ + private boolean checkIdxAlreadyExistLog(GridQueryProcessor qryProc, String idxName, String tblName, String... reqFlds) { + String msg0 = "Index with the given set or subset of columns already exists"; + + String sql = "create index \"" + idxName + "\" on " + tblName + "("; + + for (int i = 0; i < reqFlds.length; ++i) + sql += reqFlds[i] + ((i < reqFlds.length - 1) ? ", " : ")"); + + LogListener lsnrIdx = LogListener.matches(msg0).andMatches(idxName).build(); + + srvLog.registerListener(lsnrIdx); + + qryProc.querySqlFields(new SqlFieldsQuery(sql), true).getAll(); + + return lsnrIdx.check(); + } + + /** */ + private void populateTable(GridQueryProcessor qryProc, String tblName, int consPkFldsNum, String... reqFlds) { + assert consPkFldsNum <= reqFlds.length; + + String sql = "CREATE TABLE " + tblName + " ("; + + String sqlIns = "INSERT INTO " + tblName + " ("; + + for (int i = 0; i < reqFlds.length; ++i) { + sql += reqFlds[i] + " VARCHAR, "; + + sqlIns += reqFlds[i] + ((i < reqFlds.length - 1) ? ", " : ") values ("); + } + + if (consPkFldsNum > 0) { + sql += " CONSTRAINT PK_PERSON PRIMARY KEY ("; + + for (int i = 0; i < consPkFldsNum; ++i) + sql += reqFlds[i] + ((i < consPkFldsNum - 1) ? ", " : "))"); + } + else + sql += ")"; + + qryProc.querySqlFields(new SqlFieldsQuery(sql), true); + + for (int i = 0; i < 10; ++i) { + String s0 = sqlIns; + + for (int f = 0; f < reqFlds.length; ++f) + s0 += i + ((f < reqFlds.length - 1) ? ", " : ")"); + + qryProc.querySqlFields(new SqlFieldsQuery(s0), true).getAll(); + } + } + + /** + * Checks index usage for full coverage. + */ + @Test + public void testAllTableFieldsCoveredByIdx() throws Exception { + inlineSize = 10; + + srvLog = new ListeningTestLogger(false, log); + + IgniteEx ig0 = startGrid(0); + + GridQueryProcessor qryProc = ig0.context().query(); + + populateTable(qryProc, TEST_TBL_NAME, 2, "FIRST_NAME", "LAST_NAME", + "ADDRESS", "LANG"); + + assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LANG")); + + assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LAST_NAME")); + + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME")); + + assertTrue(checkIdxUsed(qryProc, "_key_PK", TEST_TBL_NAME, "FIRST_NAME", + "LAST_NAME", "LANG", "ADDRESS")); + + assertTrue(checkIdxAlreadyExistLog( + qryProc, "idx1", TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME")); + + String sqlIdx2 = String.format("create index \"idx2\" on %s(LANG, ADDRESS)", TEST_TBL_NAME); + + qryProc.querySqlFields(new SqlFieldsQuery(sqlIdx2), true).getAll(); + + assertTrue(checkIdxUsed(qryProc, "idx2", TEST_TBL_NAME, "FIRST_NAME", + "LAST_NAME", "LANG", "ADDRESS")); + + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", + "LAST_NAME", "ADDRESS", "LANG")); + // first idx fields not belongs to request fields. + assertTrue(checkIdxUsed(qryProc, "idx2", TEST_TBL_NAME, "ADDRESS", + "LAST_NAME")); + } + + /** + * Checks index usage for partial coverage. + * Last field not participate in any index. + */ + @Test + public void testPartialTableFieldsCoveredByIdx() throws Exception { + inlineSize = 10; + + String msg0 = "Index with the given set or subset of columns already exists"; + + srvLog = new ListeningTestLogger(false, log); + + IgniteEx ig0 = startGrid(0); + + GridQueryProcessor qryProc = ig0.context().query(); + + String cacheName = QueryUtils.createTableCacheName("PUBLIC", "TEST_TABLE"); + + populateTable(qryProc, TEST_TBL_NAME, 2, "FIRST_NAME", "LAST_NAME", + "ADDRESS", "LANG", "GENDER"); + + IgniteCache jcache = ig0.cache(cacheName); + + assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LANG")); + + assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LAST_NAME")); + + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME")); + + assertTrue(checkIdxUsed(qryProc, "_key_PK", TEST_TBL_NAME, "FIRST_NAME", + "LAST_NAME", "LANG", "ADDRESS")); + + assertTrue(checkIdxAlreadyExistLog( + qryProc, "idx1", TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME")); + + String sqlIdx2 = String.format("create index \"idx2\" on %s(LANG, ADDRESS)", TEST_TBL_NAME); + + qryProc.querySqlFields(new SqlFieldsQuery(sqlIdx2), true).getAll(); + // _key_PK used. + assertFalse(checkIdxUsed(qryProc, "idx2", TEST_TBL_NAME, "FIRST_NAME", + "LAST_NAME", "LANG", "ADDRESS")); + // _key_PK used. + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", + "LAST_NAME", "LANG", "ADDRESS")); + + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", + "LAST_NAME", "ADDRESS", "LANG")); + // first idx fields not belongs to request fields. + assertFalse(checkIdxUsed(qryProc, "idx2", TEST_TBL_NAME, "ADDRESS", + "LAST_NAME")); + + assertFalse(checkIdxAlreadyExistLog( + qryProc, "idx3", TEST_TBL_NAME, "ADDRESS", "LANG")); + + assertTrue(checkIdxAlreadyExistLog( + qryProc, "idx4", TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "ADDRESS", "LANG")); + + LogListener lsnrIdx4 = LogListener.matches(msg0).andMatches("_key_PK").build(); + + srvLog.registerListener(lsnrIdx4); + + String sqlIdx5 = String.format("create index \"idx5\" on %s(FIRST_NAME, LAST_NAME, LANG, ADDRESS)", TEST_TBL_NAME); + + jcache.query(new SqlFieldsQuery(sqlIdx5)).getAll(); + + assertTrue(lsnrIdx4.check()); + } + + /** + * Tests different fields sequence in indexes. + */ + @Test + public void testCreateIdxWithDifferentIdxFldsSeq() throws Exception { + inlineSize = 10; + + srvLog = new ListeningTestLogger(false, log); + + IgniteEx ig0 = startGrid(0); + + IgniteEx client = startGrid(CLIENT_NAME); + + GridQueryProcessor qryProc = ig0.context().query(); + + populateTable(qryProc, TEST_TBL_NAME, 1, "c1", "c2", "c3", "c4", "c5"); + + assertFalse(checkIdxAlreadyExistLog( + qryProc, "idx1", TEST_TBL_NAME, "c1", "c2", "c3", "c4", "c5")); + + assertFalse(checkIdxAlreadyExistLog( + qryProc, "idx2", TEST_TBL_NAME, "c1", "c3", "c4", "c5")); + + assertTrue(checkIdxAlreadyExistLog( + qryProc, "idx3", TEST_TBL_NAME, "c1", "c2")); + + assertTrue(checkIdxAlreadyExistLog( + qryProc, "idx4", TEST_TBL_NAME, "c1", "c3")); + + assertFalse(checkIdxAlreadyExistLog( + qryProc, "idx5", TEST_TBL_NAME, "c1", "c4", "c5")); + + GridQueryProcessor qryProcCl = client.context().query(); + + assertTrue(checkIdxAlreadyExistLog( + qryProcCl, "idx6", TEST_TBL_NAME, "c1", "c2")); + } + + /** + * Tests different fields sequence in indexes. + * Last field not participate in any index. + */ + @Test + public void testIndexWithDifferentFldsReqPartialFldsInIdx() throws Exception { + inlineSize = 10; + + IgniteEx ig0 = startGrid(0); + + GridQueryProcessor qryProc = ig0.context().query(); + + populateTable(qryProc, TEST_TBL_NAME, 2, "FIRST_NAME", "LAST_NAME", + "ADDRESS", "LANG", "GENDER"); + + String sqlIdx1 = String.format("create index \"idx1\" on %s(LANG, LAST_NAME, ADDRESS, FIRST_NAME)", TEST_TBL_NAME); + + qryProc.querySqlFields(new SqlFieldsQuery(sqlIdx1), true).getAll(); + + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "LANG")); + + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "ADDRESS")); + + assertFalse(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LAST_NAME", "ADDRESS")); + } + + /** + * Tests different fields sequence in indexes. + * All fields covered by indexes. + */ + @Test + public void testIndexWithDifferentFldsReqAllFldsInIdx() throws Exception { + inlineSize = 10; + + IgniteEx ig0 = startGrid(0); + + GridQueryProcessor qryProc = ig0.context().query(); + + populateTable(qryProc, TEST_TBL_NAME, 2, "FIRST_NAME", "LAST_NAME", + "ADDRESS", "LANG"); + + String sqlIdx1 = String.format("create index \"idx1\" on %s(LANG, LAST_NAME, ADDRESS, FIRST_NAME)", TEST_TBL_NAME); + + qryProc.querySqlFields(new SqlFieldsQuery(sqlIdx1), true).getAll(); + + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "LANG")); + + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME", "ADDRESS")); + + assertTrue(checkIdxUsed(qryProc, null, TEST_TBL_NAME, "LAST_NAME", "ADDRESS")); + } + /** */ @Test public void testNoIndexesWithPersistence() throws Exception { From 2075cb657897056f45eea4f6f64a9efc71deea59 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 19:18:47 +0300 Subject: [PATCH 04/69] IGNITE-10619: revert changes from master 2 --- .../processors/query/QueryIndexDescriptorImpl.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java index 1b85af579f8c6..8e54261a5b09c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java @@ -17,17 +17,16 @@ package org.apache.ignite.internal.processors.query; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.cache.QueryIndexType; -import org.apache.ignite.internal.util.tostring.GridToStringExclude; -import org.apache.ignite.internal.util.typedef.T2; -import org.apache.ignite.internal.util.typedef.internal.S; - import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; import java.util.HashSet; import java.util.TreeSet; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.QueryIndexType; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.S; /** * Index descriptor. @@ -127,7 +126,7 @@ public QueryIndexDescriptorImpl addField(String field, int orderNum, boolean des if (descending) { if (descendings == null) - descendings = new HashSet<>(); + descendings = new HashSet<>(); descendings.add(field); } From 6fe880076aba76f3b2235bf8643a6f3286c8f564 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 19:19:42 +0300 Subject: [PATCH 05/69] IGNITE-10619: revert changes from master 3 --- .../ignite/internal/processors/query/h2/opt/GridH2Table.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java index 666d712190ecf..a4dfa16f81ba3 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java @@ -31,12 +31,13 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; import java.util.stream.Stream; + import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteInterruptedException; -import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.query.QueryRetryException; -import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheContextInfo; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; From a176c1f891a61299fe69c3a4352ed212436e6023 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 19:22:57 +0300 Subject: [PATCH 06/69] IGNITE-10619: revert changes from master 4 --- .../processors/query/QueryIndexDescriptorImpl.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java index 8e54261a5b09c..d8b972cdbbec3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java @@ -17,17 +17,18 @@ package org.apache.ignite.internal.processors.query; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Comparator; -import java.util.HashSet; -import java.util.TreeSet; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.QueryIndexType; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.S; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashSet; +import java.util.TreeSet; + /** * Index descriptor. */ From f25c7c302451c2c5b28e845c5fdef88ef59d3740 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 19:32:03 +0300 Subject: [PATCH 07/69] IGNITE-10619: move methods from session to selector session --- .../internal/util/nio/GridNioSessionImpl.java | 17 ----------------- .../util/nio/GridSelectorNioSessionImpl.java | 17 +++++++++++++++++ 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java index 89ece47017c70..51cb5581d5d5a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java @@ -80,9 +80,6 @@ public class GridNioSessionImpl implements GridNioSession { /** For debug purposes. */ private volatile boolean markedForClose; - /** Close channel on session #close() called. */ - private volatile boolean closeSocket = true; - /** * @param filterChain Chain. * @param locAddr Local address. @@ -327,20 +324,6 @@ public boolean setClosed() { return closeTime.compareAndSet(0, U.currentTimeMillis()); } - /** - * @return {@code True} to close SocketChannel on current session close occured. - */ - public boolean closeSocketOnSessionClose() { - return closeSocket; - } - - /** - * @param closeSocket {@code False} remain SocketChannel open on session close. - */ - public void closeSocketOnSessionClose(boolean closeSocket) { - this.closeSocket = closeSocket; - } - /** * @return {@code True} if this session was closed. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java index 11fc6fc83a3b2..ac2fc6133d54c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java @@ -78,6 +78,9 @@ public class GridSelectorNioSessionImpl extends GridNioSessionImpl implements Gr /** */ private Object sysMsg; + /** Close channel on session #close() called. */ + private volatile boolean closeSocket = true; + /** * Creates session instance. * @@ -180,6 +183,20 @@ public SelectionKey key() { return key; } + /** + * @return {@code True} to close SocketChannel on current session close occured. + */ + public boolean closeSocketOnSessionClose() { + return closeSocket; + } + + /** + * @param closeSocket {@code False} remain SocketChannel open on session close. + */ + public void closeSocketOnSessionClose(boolean closeSocket) { + this.closeSocket = closeSocket; + } + /** * @param from Current session worker. * @param fut Move future. From 9f73bdea02c50086a95c344fc47a228e741a4b4c Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 19:54:38 +0300 Subject: [PATCH 08/69] IGNITE-10619: change connection policy impl --- .../tcp/TcpCommunicationSpi.java | 35 ++++++++++++------- 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 311242e2d784c..23868fe4e9be2 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -411,8 +411,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati /** */ private ConnectionPolicy connPlc = new FirstConnectionPolicy(); - /** */ - private ConnectionPolicy sockConnPlc = new ChannelRandomConnectionPolicy(); + /** Channel connection index provider. */ + private ConnectionPolicy chConnPlc; /** */ private boolean enableForcibleNodeKill = IgniteSystemProperties @@ -2248,10 +2248,13 @@ private void dumpInfo(StringBuilder sb, UUID dstNodeId) { } if (connectionsPerNode > 1) - connPlc = new RoundRobinConnectionPolicy(); + connPlc = new RoundRobinConnectionPolicy(0, connectionsPerNode); else connPlc = new FirstConnectionPolicy(); + chConnPlc = new RoundRobinConnectionPolicy(connectionsPerNode + 1, + MAX_CHANNEL_CONN_PER_NODE); + try { locHost = U.resolveLocalHost(locAddr); } @@ -4378,10 +4381,11 @@ public IgniteInternalFuture openChannel( ) throws IgniteSpiException { assert !remote.isLocal() : remote; assert initMsg != null; + assert chConnPlc != null; assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direct connection over socket channel " + "[nodeId=" + remote.id() + ']'; - ConnectionKey key = new ConnectionKey(remote.id(), sockConnPlc.connectionIndex()); + ConnectionKey key = new ConnectionKey(remote.id(), chConnPlc.connectionIndex()); GridFutureAdapter result; @@ -5105,18 +5109,25 @@ private static class FirstConnectionPolicy implements ConnectionPolicy { } /** */ - private class RoundRobinConnectionPolicy implements ConnectionPolicy { - /** {@inheritDoc} */ - @Override public int connectionIndex() { - return (int)(U.safeAbs(Thread.currentThread().getId()) % connectionsPerNode); + private static class RoundRobinConnectionPolicy implements ConnectionPolicy { + /** Position to start index at. */ + private final int startIdx; + + /** Max distinct number of indexes to produce. */ + private final int maxConn; + + /** + * @param startIdx Position to start index at. + * @param maxConn Max distinct number of indexes to produce. + */ + public RoundRobinConnectionPolicy(int startIdx, int maxConn) { + this.startIdx = startIdx; + this.maxConn = maxConn; } - } - /** */ - private static class ChannelRandomConnectionPolicy implements ConnectionPolicy { /** {@inheritDoc} */ @Override public int connectionIndex() { - return MAX_CONN_PER_NODE + (int)(U.safeAbs(Thread.currentThread().getId() % MAX_CHANNEL_CONN_PER_NODE)); + return startIdx + (int)(U.safeAbs(Thread.currentThread().getId()) % maxConn); } } From f919ec9e378d25ed6322ecc7c533e0144f4a6055 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 20:03:52 +0300 Subject: [PATCH 09/69] IGNITE-10619: change communication listener type --- .../spi/communication/tcp/TcpCommunicationSpi.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 23868fe4e9be2..c4e81c40ddd0a 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -1285,7 +1285,7 @@ class ConnectClosure implements IgniteInClosure { private final ConcurrentMap> channelReqs = new ConcurrentHashMap<>(); /** SPI listener. */ - private volatile CommunicationListener lsnr; + private volatile CommunicationListenerEx lsnr; /** Bound port. */ private int boundTcpPort = -1; @@ -1998,7 +1998,7 @@ public TcpCommunicationSpi setSlowClientQueueLimit(int slowClientQueueLimit) { /** {@inheritDoc} */ @Override public void setListener(CommunicationListener lsnr) { - this.lsnr = lsnr; + this.lsnr = (CommunicationListenerEx)lsnr; } /** @@ -4072,10 +4072,10 @@ private void notifyChannelEvtListener(UUID nodeId, Channel channel, Message init if (log.isDebugEnabled()) log.debug("Notify corresponding listeners due to the new channel opened: " + channel); - CommunicationListener lsnr0 = lsnr; + CommunicationListenerEx lsnr0 = lsnr; - if (lsnr0 instanceof CommunicationListenerEx) - ((CommunicationListenerEx)lsnr0).onChannelOpened(nodeId, initMsg, channel); + if (lsnr0 != null) + lsnr0.onChannelOpened(nodeId, initMsg, channel); } /** From 9d21fea81b866cf03e07912e7667f0a9d962d956 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 20:43:30 +0300 Subject: [PATCH 10/69] IGNITE-10619: remove final session field --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index c4e81c40ddd0a..4edecef7ef63d 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -4401,8 +4401,6 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc assert ses != null : "Session must be established [remoteId=" + remote.id() + ", key=" + key + ']'; - final GridNioSession finalSes = ses; - channelReqs.put(key, result = new GridFutureAdapter<>()); // Send configuration message over the created session. @@ -4421,7 +4419,7 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc @Override public void onTimeout() { // Close session if request not complete yet. if (result.onDone(handshakeTimeoutException())) - finalSes.close(); + ses.close(); } }); } From 0d36b081556707ea482e6c532db22807c125a25f Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 20:46:06 +0300 Subject: [PATCH 11/69] IGNITE-10619: code style changes --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 4edecef7ef63d..1d3bd5986279c 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -4406,7 +4406,9 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc // Send configuration message over the created session. ses.send(new ChannelCreateRequest(initMsg)) .listen(f -> { - if (f.error() == null) { + if (f.error() != null) + result.onDone(f.error()); + else { addTimeoutObject(new IgniteSpiTimeoutObject() { @Override public IgniteUuid id() { return IgniteUuid.randomUuid(); @@ -4423,8 +4425,6 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc } }); } - else - result.onDone(f.error()); }); return result; From 3959251b1f3405784092472c362ed31ca41b4bfa Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 20:49:48 +0300 Subject: [PATCH 12/69] IGNITE-10619: change connection timeout --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 1d3bd5986279c..0c14c677bf5c8 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -4415,7 +4415,7 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc } @Override public long endTime() { - return U.currentTimeMillis() + DFLT_CONN_TIMEOUT; + return U.currentTimeMillis() + connTimeout; } @Override public void onTimeout() { From 6af11c2b835a2b192028adc7b6d1a4e05c5af655 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 22:15:28 +0300 Subject: [PATCH 13/69] IGNITE-10619: change connection policy --- .../managers/communication/GridIoManager.java | 1 + .../tcp/TcpCommunicationSpi.java | 37 +++++++++++-------- 2 files changed, 23 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 67ff6bff1ad60..ef4134c4cddb7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2663,6 +2663,7 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa ", fromNodeId=" + nodeId + ']')); out.writeObject(new TransmissionMeta(ex)); + out.flush(); return; } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 0c14c677bf5c8..e2c172507b072 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -50,6 +50,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import javax.net.ssl.SSLEngine; import javax.net.ssl.SSLException; import org.apache.ignite.Ignite; @@ -383,9 +384,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati /** Maximum {@link GridNioSession} connections per node. */ public static final int MAX_CONN_PER_NODE = 1024; - /** Maximum {@link Channel} connections per node. */ - public static final int MAX_CHANNEL_CONN_PER_NODE = 256; - /** No-op runnable. */ private static final IgniteRunnable NOOP = new IgniteRunnable() { @Override public void run() { @@ -772,8 +770,6 @@ private void handleChannelCreateResponse(GridSelectorNioSessionImpl ses, Connect try { f.get(); // Exception not ocurred. - cleanupLocalNodeRecoveryDescriptor(connKey); - SelectableChannel nioChannel = ses.key().channel(); reqFut.onDone(nioChannel); @@ -2252,8 +2248,14 @@ private void dumpInfo(StringBuilder sb, UUID dstNodeId) { else connPlc = new FirstConnectionPolicy(); - chConnPlc = new RoundRobinConnectionPolicy(connectionsPerNode + 1, - MAX_CHANNEL_CONN_PER_NODE); + chConnPlc = new ConnectionPolicy() { + /** Sequential connection index provider. */ + private final AtomicInteger connIdx = new AtomicInteger(connectionsPerNode + 1); + + @Override public int connectionIndex() { + return connIdx.incrementAndGet(); + } + }; try { locHost = U.resolveLocalHost(locAddr); @@ -4387,27 +4389,32 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc ConnectionKey key = new ConnectionKey(remote.id(), chConnPlc.connectionIndex()); - GridFutureAdapter result; + if (channelReqs.get(key) != null) { + throw new IgniteSpiException("The channel connection cannot be established to remote node. " + + "Connection key already in use [key=" + key + ']'); + } + + GridFutureAdapter result = new GridFutureAdapter<>(); connectGate.enter(); try { - if (channelReqs.get(key) != null) { - throw new IgniteSpiException("The channel connection cannot be established to remote node. " + - "Connection key already in use [key=" + key + ']'); - } - GridNioSession ses = createNioSession(remote, key.connectionIndex()); + cleanupLocalNodeRecoveryDescriptor(key); + assert ses != null : "Session must be established [remoteId=" + remote.id() + ", key=" + key + ']'; - channelReqs.put(key, result = new GridFutureAdapter<>()); + channelReqs.put(key, result); // Send configuration message over the created session. ses.send(new ChannelCreateRequest(initMsg)) .listen(f -> { - if (f.error() != null) + if (f.error() != null) { result.onDone(f.error()); + + ses.close(); + } else { addTimeoutObject(new IgniteSpiTimeoutObject() { @Override public IgniteUuid id() { From b4197996f105ca03f51eceff0d75e7c3ea2a4f18 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 22:17:19 +0300 Subject: [PATCH 14/69] IGNITE-10619: revert change of roundrobin connection policy --- .../tcp/TcpCommunicationSpi.java | 19 ++----------------- 1 file changed, 2 insertions(+), 17 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index e2c172507b072..b5a9ed2068ad2 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -5114,25 +5114,10 @@ private static class FirstConnectionPolicy implements ConnectionPolicy { } /** */ - private static class RoundRobinConnectionPolicy implements ConnectionPolicy { - /** Position to start index at. */ - private final int startIdx; - - /** Max distinct number of indexes to produce. */ - private final int maxConn; - - /** - * @param startIdx Position to start index at. - * @param maxConn Max distinct number of indexes to produce. - */ - public RoundRobinConnectionPolicy(int startIdx, int maxConn) { - this.startIdx = startIdx; - this.maxConn = maxConn; - } - + private class RoundRobinConnectionPolicy implements ConnectionPolicy { /** {@inheritDoc} */ @Override public int connectionIndex() { - return startIdx + (int)(U.safeAbs(Thread.currentThread().getId()) % maxConn); + return (int)(U.safeAbs(Thread.currentThread().getId()) % connectionsPerNode); } } From eb3bc7aaaa0661c4cc1642a82e856a4e088cb62e Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 22:17:40 +0300 Subject: [PATCH 15/69] IGNITE-10619: revert change of roundrobin connection policy 2 --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index b5a9ed2068ad2..5d6c45dbfbbe7 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -2244,7 +2244,7 @@ private void dumpInfo(StringBuilder sb, UUID dstNodeId) { } if (connectionsPerNode > 1) - connPlc = new RoundRobinConnectionPolicy(0, connectionsPerNode); + connPlc = new RoundRobinConnectionPolicy(); else connPlc = new FirstConnectionPolicy(); From a5a55a1b84f89a6df9f967fedc9cc3fecfbd50cf Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 22:19:31 +0300 Subject: [PATCH 16/69] IGNITE-10619: cleanup local requests --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 5d6c45dbfbbe7..0d1ab18677218 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -785,6 +785,8 @@ private void handleChannelCreateRequest( ConnectionKey connKey, ChannelCreateRequest msg ) { + cleanupLocalNodeRecoveryDescriptor(connKey); + ses.send(new ChannelCreateResponse()) .listen(sendFut -> { try { @@ -797,8 +799,6 @@ private void handleChannelCreateRequest( try { closeFut.get(); // Exception not ocurred. - cleanupLocalNodeRecoveryDescriptor(connKey); - SelectableChannel channel = ses.key().channel(); notifyChannelEvtListener(connKey.nodeId(), channel, msg.message()); From 7bbd7b0feb4f2900fab5db80066d126bfd9e1a51 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 22:25:26 +0300 Subject: [PATCH 17/69] IGNITE-10619: remove channel requests --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 0d1ab18677218..01075f9ce7a69 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -4413,6 +4413,8 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc if (f.error() != null) { result.onDone(f.error()); + channelReqs.remove(key); + ses.close(); } else { @@ -4427,8 +4429,11 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc @Override public void onTimeout() { // Close session if request not complete yet. - if (result.onDone(handshakeTimeoutException())) + if (result.onDone(handshakeTimeoutException())) { ses.close(); + + channelReqs.remove(key); + } } }); } From a39df5fef1412fb4d89b78ba1a963e11d610bfb1 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 1 Aug 2019 22:32:26 +0300 Subject: [PATCH 18/69] IGNITE-10619: code style changes --- .../spi/communication/tcp/TcpCommunicationSpi.java | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 01075f9ce7a69..8b54601db33e4 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -767,16 +767,10 @@ private void handleChannelCreateResponse(GridSelectorNioSessionImpl ses, Connect ses.closeSocketOnSessionClose(false); ses.close().listen(f -> { - try { - f.get(); // Exception not ocurred. - - SelectableChannel nioChannel = ses.key().channel(); - - reqFut.onDone(nioChannel); - } - catch (IgniteCheckedException e) { - reqFut.onDone(e); - } + if (f.error() == null) + reqFut.onDone(ses.key().channel()); + else + reqFut.onDone(f.error()); }); } From 67f2d545b71d15aeaca7ae2411eed69c53cd1c27 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 2 Aug 2019 12:07:23 +0300 Subject: [PATCH 19/69] IGNITE-10619: code simplify on tcp communication spi --- .../tcp/TcpCommunicationSpi.java | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 8b54601db33e4..eb4c20ce6008d 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -783,33 +783,28 @@ private void handleChannelCreateRequest( ses.send(new ChannelCreateResponse()) .listen(sendFut -> { - try { - sendFut.get(); // Exception not ocurred. - + if (sendFut.error() == null) { ses.closeSocketOnSessionClose(false); // Close session and send response. ses.close().listen(closeFut -> { - try { - closeFut.get(); // Exception not ocurred. - - SelectableChannel channel = ses.key().channel(); - - notifyChannelEvtListener(connKey.nodeId(), channel, msg.message()); - } - catch (IgniteCheckedException e) { + if (closeFut.error() == null) + notifyChannelEvtListener(connKey.nodeId(), ses.key().channel(), msg.message()); + else { U.error(log, "Nio session has not been properly closed " + - "[nodeId=" + connKey.nodeId() + ", idx=" + connKey.connectionIndex() + ']', e); + "[nodeId=" + connKey.nodeId() + ", idx=" + connKey.connectionIndex() + ']', + closeFut.error()); ses.closeSocketOnSessionClose(true); + U.closeQuiet(ses.key().channel()); } }); } - catch (IgniteCheckedException e) { + else { U.error(log, "Fail to send channel creation response to the remote node. " + "Session will be closed [nodeId=" + connKey.nodeId() + - ", idx=" + connKey.connectionIndex() + ']', e); + ", idx=" + connKey.connectionIndex() + ']', sendFut.error()); ses.close(); } From baef9eb48bf2202412de0fe4894936c6214ff1fc Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 2 Aug 2019 12:12:02 +0300 Subject: [PATCH 20/69] IGNITE-10619: javadoc changes --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index eb4c20ce6008d..266084cd3bbe0 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -1266,7 +1266,7 @@ class ConnectClosure implements IgniteInClosure { /** Clients. */ private final ConcurrentMap clients = GridConcurrentFactory.newMap(); - /** Java NIO channels. */ + /** Channel creation local requests (registered on #openChannel()) */ private final ConcurrentMap> channelReqs = new ConcurrentHashMap<>(); /** SPI listener. */ @@ -4061,7 +4061,7 @@ else if (log.isDebugEnabled()) */ private void notifyChannelEvtListener(UUID nodeId, Channel channel, Message initMsg) { if (log.isDebugEnabled()) - log.debug("Notify corresponding listeners due to the new channel opened: " + channel); + log.debug("Notify appropriate listeners due to a new channel opened: " + channel); CommunicationListenerEx lsnr0 = lsnr; From f071d139ae30e92d6b6bcbe53b5218e2857fc21e Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 2 Aug 2019 13:11:16 +0300 Subject: [PATCH 21/69] IGNITE-10619: reformat code internals --- .../tcp/TcpCommunicationSpi.java | 102 ++++++++++-------- 1 file changed, 59 insertions(+), 43 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 266084cd3bbe0..61a325e710947 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -761,16 +761,23 @@ private void handleChannelCreateResponse(GridSelectorNioSessionImpl ses, Connect "response message. Message will be ignored [remoteId=" + connKey.nodeId() + ", idx=" + connKey.connectionIndex() + ']'); + ses.close(); + + assert cleanupLocalNodeRecoveryDescriptor(connKey) : connKey; + return; } ses.closeSocketOnSessionClose(false); ses.close().listen(f -> { - if (f.error() == null) - reqFut.onDone(ses.key().channel()); - else + if (f.error() != null) { reqFut.onDone(f.error()); + + return; + } + + reqFut.onDone(ses.key().channel()); }); } @@ -783,31 +790,34 @@ private void handleChannelCreateRequest( ses.send(new ChannelCreateResponse()) .listen(sendFut -> { - if (sendFut.error() == null) { - ses.closeSocketOnSessionClose(false); - - // Close session and send response. - ses.close().listen(closeFut -> { - if (closeFut.error() == null) - notifyChannelEvtListener(connKey.nodeId(), ses.key().channel(), msg.message()); - else { - U.error(log, "Nio session has not been properly closed " + - "[nodeId=" + connKey.nodeId() + ", idx=" + connKey.connectionIndex() + ']', - closeFut.error()); - - ses.closeSocketOnSessionClose(true); - - U.closeQuiet(ses.key().channel()); - } - }); - } - else { + if (sendFut.error() != null) { U.error(log, "Fail to send channel creation response to the remote node. " + "Session will be closed [nodeId=" + connKey.nodeId() + ", idx=" + connKey.connectionIndex() + ']', sendFut.error()); ses.close(); + + return; } + + ses.closeSocketOnSessionClose(false); + + // Close session and send response. + ses.close().listen(closeFut -> { + if (closeFut.error() != null) { + U.error(log, "Nio session has not been properly closed " + + "[nodeId=" + connKey.nodeId() + ", idx=" + connKey.connectionIndex() + ']', + closeFut.error()); + + ses.closeSocketOnSessionClose(true); + + U.closeQuiet(ses.key().channel()); + + return; + } + + notifyChannelEvtListener(connKey.nodeId(), ses.key().channel(), msg.message()); + }); }); } @@ -4156,16 +4166,21 @@ private boolean usePairedConnections(ClusterNode node) { /** * @param key The connection key to cleanup descriptors on local node. + * @return {@code true} if there is no descriptors for the given {@code key}. */ - private void cleanupLocalNodeRecoveryDescriptor(ConnectionKey key) { + private boolean cleanupLocalNodeRecoveryDescriptor(ConnectionKey key) { ClusterNode node = getLocalNode(); - if (usePairedConnections(node)){ - inRecDescs.remove(key); - outRecDescs.remove(key); + boolean empty = true; + + if (usePairedConnections(node)) { + empty &= inRecDescs.remove(key) == null; + empty &= outRecDescs.remove(key) == null; } else - recoveryDescs.remove(key); + empty &= recoveryDescs.remove(key) == null; + + return empty; } /** @@ -4405,27 +4420,28 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc channelReqs.remove(key); ses.close(); + + return; } - else { - addTimeoutObject(new IgniteSpiTimeoutObject() { - @Override public IgniteUuid id() { - return IgniteUuid.randomUuid(); - } - @Override public long endTime() { - return U.currentTimeMillis() + connTimeout; - } + addTimeoutObject(new IgniteSpiTimeoutObject() { + @Override public IgniteUuid id() { + return IgniteUuid.randomUuid(); + } - @Override public void onTimeout() { - // Close session if request not complete yet. - if (result.onDone(handshakeTimeoutException())) { - ses.close(); + @Override public long endTime() { + return U.currentTimeMillis() + connTimeout; + } - channelReqs.remove(key); - } + @Override public void onTimeout() { + // Close session if request not complete yet. + if (result.onDone(handshakeTimeoutException())) { + ses.close(); + + channelReqs.remove(key); } - }); - } + } + }); }); return result; From 842157c3d8cefae04c4c3b648f6af0a657326db7 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 2 Aug 2019 13:49:17 +0300 Subject: [PATCH 22/69] IGNITE-10619: remove assertion of cleanup --- .../communication/tcp/TcpCommunicationSpi.java | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 61a325e710947..702fa65f10698 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -763,8 +763,6 @@ private void handleChannelCreateResponse(GridSelectorNioSessionImpl ses, Connect ses.close(); - assert cleanupLocalNodeRecoveryDescriptor(connKey) : connKey; - return; } @@ -4166,21 +4164,16 @@ private boolean usePairedConnections(ClusterNode node) { /** * @param key The connection key to cleanup descriptors on local node. - * @return {@code true} if there is no descriptors for the given {@code key}. */ - private boolean cleanupLocalNodeRecoveryDescriptor(ConnectionKey key) { + private void cleanupLocalNodeRecoveryDescriptor(ConnectionKey key) { ClusterNode node = getLocalNode(); - boolean empty = true; - if (usePairedConnections(node)) { - empty &= inRecDescs.remove(key) == null; - empty &= outRecDescs.remove(key) == null; + inRecDescs.remove(key); + outRecDescs.remove(key); } else - empty &= recoveryDescs.remove(key) == null; - - return empty; + recoveryDescs.remove(key); } /** From 794bccc3f654a0ef6a6cf58113df8bd7741339a9 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 2 Aug 2019 13:50:26 +0300 Subject: [PATCH 23/69] IGNITE-10619: remove assertion of cleanup 2 --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 702fa65f10698..4247bef9da739 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -807,8 +807,6 @@ private void handleChannelCreateRequest( "[nodeId=" + connKey.nodeId() + ", idx=" + connKey.connectionIndex() + ']', closeFut.error()); - ses.closeSocketOnSessionClose(true); - U.closeQuiet(ses.key().channel()); return; From 3264e777fd8048cda8bee2b5b137be1a374e3f4d Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 2 Aug 2019 16:34:31 +0300 Subject: [PATCH 24/69] IGNITE-10619: fix removing channel create request --- .../communication/tcp/TcpCommunicationSpi.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 4247bef9da739..342473df5d4ea 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -4406,9 +4406,11 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc ses.send(new ChannelCreateRequest(initMsg)) .listen(f -> { if (f.error() != null) { - result.onDone(f.error()); + GridFutureAdapter rq = channelReqs.remove(key); - channelReqs.remove(key); + assert rq != null; + + rq.onDone(f.error()); ses.close(); @@ -4426,11 +4428,13 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc @Override public void onTimeout() { // Close session if request not complete yet. - if (result.onDone(handshakeTimeoutException())) { - ses.close(); + GridFutureAdapter rq = channelReqs.remove(key); - channelReqs.remove(key); - } + if (rq == null) + return; + + if (rq.onDone(handshakeTimeoutException())) + ses.close(); } }); }); From cf898f7f40f43a1fbcbda9268a5d743d6f6e44ea Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 2 Aug 2019 17:08:50 +0300 Subject: [PATCH 25/69] IGNITE-10619: minor code changes --- .../managers/communication/FileSender.java | 15 +++++++-------- .../communication/tcp/TcpCommunicationSpi.java | 4 ++-- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 5441a0dd470d3..66f0c69a0e66b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -49,7 +49,7 @@ class FileSender extends AbstractTransmission { /** Default factory to provide IO oprations over given file. */ @GridToStringExclude - private final FileIOFactory fileIoFactory; + private final FileIOFactory factory; /** File which will be send to remote by chunks. */ private final File file; @@ -60,7 +60,7 @@ class FileSender extends AbstractTransmission { /** * @param file File which is going to be send by chunks. - * @param pos File offset. + * @param off File offset. * @param cnt Number of bytes to transfer. * @param params Additional file params. * @param stopChecker Node stop or prcoess interrupt checker. @@ -70,7 +70,7 @@ class FileSender extends AbstractTransmission { */ public FileSender( File file, - long pos, + long off, long cnt, Map params, BooleanSupplier stopChecker, @@ -78,7 +78,7 @@ public FileSender( FileIOFactory factory, int chunkSize ) { - super(new TransmissionMeta(file.getName(), pos, cnt, params, null, null), + super(new TransmissionMeta(file.getName(), off, cnt, params, null, null), stopChecker, log, chunkSize); @@ -86,7 +86,7 @@ public FileSender( assert file != null; this.file = file; - fileIoFactory = factory; + this.factory = factory; } /** @@ -105,7 +105,7 @@ public void send(WritableByteChannel ch, try { // Can be not null if reconnection is going to be occurred. if (fileIo == null) - fileIo = fileIoFactory.create(file); + fileIo = factory.create(file); } catch (IOException e) { // Consider this IO exeption as a user one (not the network exception) and interrupt upload process. @@ -145,9 +145,8 @@ public void send(WritableByteChannel ch, /** * @param connMeta Conneciton meta info. - * @throws IgniteCheckedException If fails. */ - private void state(TransmissionMeta connMeta) throws IgniteCheckedException { + private void state(TransmissionMeta connMeta) { assert connMeta != null; assert fileIo != null; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 342473df5d4ea..77e605616ab8a 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -4396,10 +4396,10 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc try { GridNioSession ses = createNioSession(remote, key.connectionIndex()); - cleanupLocalNodeRecoveryDescriptor(key); - assert ses != null : "Session must be established [remoteId=" + remote.id() + ", key=" + key + ']'; + cleanupLocalNodeRecoveryDescriptor(key); + channelReqs.put(key, result); // Send configuration message over the created session. From 008ef069c74ab144421a1ee153072056d5f32791 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 2 Aug 2019 17:14:55 +0300 Subject: [PATCH 26/69] IGNITE-10619: javadoc fixes --- .../ignite/internal/managers/communication/FileSender.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 66f0c69a0e66b..bad8c5fa14e53 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -116,7 +116,7 @@ public void send(WritableByteChannel ch, if (connMeta != null) state(connMeta); - // Write to remote about transission `is in active` mode. + // Write flag to remote to keep currnet transmission opened. oo.writeBoolean(false); // Send meta about current file to remote. @@ -150,7 +150,7 @@ private void state(TransmissionMeta connMeta) { assert connMeta != null; assert fileIo != null; - // Remote note doesn't have file info. + // The remote node doesn't have a file meta info. if (connMeta.offset() < 0) return; From 63b0d266bac9dd172123752643e508c1b93fcd45 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sat, 3 Aug 2019 23:07:37 +0300 Subject: [PATCH 27/69] IGNITE-10619: fix synchronization issues --- .../communication/AbstractReceiver.java | 2 +- .../managers/communication/FileSender.java | 7 +- .../managers/communication/GridIoManager.java | 185 ++++++++++-------- ...GridIoManagerFileTransmissionSelfTest.java | 13 +- 4 files changed, 117 insertions(+), 90 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java index fb933f27de588..e62d71443c4b2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java @@ -90,7 +90,7 @@ public void receive( public TransmissionMeta state() { return new TransmissionMeta(initMeta.name(), initMeta.offset() + transferred, - initMeta.count(), + initMeta.count() - transferred, initMeta.params(), policy(), null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index bad8c5fa14e53..cdc53a37b69cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -120,15 +120,12 @@ public void send(WritableByteChannel ch, oo.writeBoolean(false); // Send meta about current file to remote. - new TransmissionMeta(initMeta.name(), + oo.writeObject(new TransmissionMeta(initMeta.name(), initMeta.offset() + transferred, initMeta.count() - transferred, initMeta.params(), plc, - null) - .writeExternal(oo); - - oo.flush(); + null)); while (hasNextChunk()) { if (Thread.currentThread().isInterrupted() || stopped()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 119411c28795d..38bb5acb1dc31 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -41,6 +41,7 @@ import java.util.Date; import java.util.Deque; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -922,18 +923,18 @@ private void format(StringBuilder b, Collection> pairs, SimpleD } // Clear the context on the uploader node left. - for (Map.Entry sesEntry : rcvCtxs.entrySet()) { - ReceiverContext ioctx = sesEntry.getValue(); + Iterator> it = rcvCtxs.entrySet().iterator(); - if (ioctx.nodeId.equals(nodeId)) { - ioctx.hnd.onException(nodeId, - new ClusterTopologyCheckedException("Failed to proceed download. " + - "The remote node node left the grid: " + nodeId)); + while(it.hasNext()) { + Map.Entry e = it.next(); - ioctx.interrupted = true; - U.closeQuiet(ioctx.lastRcv); + if (nodeId.equals(e.getValue().nodeId)) { + it.remove(); - rcvCtxs.remove(sesEntry.getKey()); + closeRecevier(e.getValue(), + nodeId, + new ClusterTopologyCheckedException("Remove node left the grid. " + + "Receiver has been stopped : " + nodeId)); } } } @@ -1868,6 +1869,10 @@ public void addTransmissionHandler(Object topic, TransmissionHandler hnd) { */ public void removeTransmissionHandler(Object topic) { topicTransmissionHnds.remove(topic); + + closeRecevier(rcvCtxs.remove(topic), + ctx.localNodeId(), + new IgniteCheckedException("Receiver has been closed due to removing corresponding transmission handler")); } /** @@ -2654,23 +2659,43 @@ public int getOutboundMessagesQueueSize() { return getSpi().getOutboundMessagesQueueSize(); } + /** + * @param ctx Receiver context to use. + * @param nodeId Node id caused receiver to close. + * @param ex Exception to close receiver with. + */ + private static void closeRecevier(ReceiverContext ctx, UUID nodeId, IgniteCheckedException ex) { + if (ctx == null) + return; + + ctx.interrupted = true; + + U.closeQuiet(ctx.lastRcv); + + ctx.hnd.onException(nodeId, ex); + } + /** * @param topic Topic to which the channel is created. * @param nodeId Remote node id. * @param initMsg Channel initialization message with additional params. - * @param channel Channel instance. + * @param ch Channel instance. */ - private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessage initMsg, SocketChannel channel) { + private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessage initMsg, SocketChannel ch) { ReceiverContext rcvCtx = null; - IgniteUuid newSesId = null; ObjectInputStream in = null; ObjectOutputStream out = null; try { + if (stopping) { + throw new NodeStoppingException("Local node is stopping. Channel will be closed [topic=" + topic + + ", channel=" + ch + ']'); + } + TransmissionHandler hnd = topicTransmissionHnds.get(topic); if (hnd == null) { - U.warn(log, "There is no handler for given topic. Opened channel will be closed [nodeId=" + nodeId + + U.warn(log, "There is no handler for a given topic. Channel will be closed [nodeId=" + nodeId + ", topic=" + topic + ']'); return; @@ -2678,80 +2703,74 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa if (initMsg == null || initMsg.sesId() == null) { U.warn(log, "There is no initial message provied for given topic. Opened channel will be closed " + - "[nodeId=" + nodeId + ", topic=" + topic + ']'); + "[nodeId=" + nodeId + ", topic=" + topic + ", initMsg=" + initMsg + ']'); return; } - rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(nodeId, hnd)); + configureChannel(ctx.config(), ch); - configureChannel(ctx.config(), channel); + in = new ObjectInputStream(ch.socket().getInputStream()); + out = new ObjectOutputStream(ch.socket().getOutputStream()); - in = new ObjectInputStream(channel.socket().getInputStream()); - out = new ObjectOutputStream(channel.socket().getOutputStream()); + if (log.isDebugEnabled()) { + log.debug("Trasmission opens a new channel [nodeId=" + nodeId + ", topic=" + topic + + ", initMsg=" + initMsg + ']'); + } - // Do not allow multiple connection for the same session id; - if (!rcvCtx.inProgress.compareAndSet(false, true)) { - IgniteCheckedException ex; + IgniteUuid newSesId = initMsg.sesId(); - U.warn(log, ex = new IgniteCheckedException("Current topic is already being handled by " + - "another thread. Channel will be closed [initMsg=" + initMsg + ", channel=" + channel + - ", fromNodeId=" + nodeId + ']')); + synchronized (rcvCtxs) { + rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(nodeId, hnd, newSesId)); - out.writeObject(new TransmissionMeta(ex)); - out.flush(); + // Do not allow multiple connection for the same session + boolean activated = rcvCtx.active.compareAndSet(false, true); - return; - } + if (!activated && newSesId.equals(rcvCtx.sesId)) { + IOException e = new IOException("Receiver has not completed yet previous data processing. " + + "Wait for the next connection attempt."); - if (!busyLock.readLock().tryLock()) - return; + U.warn(log, e); - try { - newSesId = initMsg.sesId(); + out.writeObject(new TransmissionMeta(e)); - if (rcvCtx.sesId == null) - rcvCtx.sesId = newSesId; - else if (!rcvCtx.sesId.equals(newSesId)) { - // Attempt to receive file with new session id. Context must be reinited, - // previous session must be failed. - rcvCtx.hnd.onException(nodeId, new IgniteCheckedException("The handler has been aborted " + - "by transfer attempt with a new sessionId: " + newSesId)); + return; + } + else if (!activated && !newSesId.equals(rcvCtx.sesId)) { + IgniteCheckedException ex = new IgniteCheckedException("Receivers topic is busy by another transmission. " + + "It's not allowed to process different sessions over the same topic simultaneously. " + + "Channel will be closed [initMsg=" + initMsg + ", channel=" + ch + ", nodeId=" + nodeId + ']'); - rcvCtx = new ReceiverContext(nodeId, hnd); - rcvCtx.sesId = newSesId; - rcvCtx.inProgress.set(true); + U.error(log, ex); - rcvCtxs.put(topic, rcvCtx); - } + out.writeObject(new TransmissionMeta(ex)); - // Send previous context state to sync remote and local node (on manager connected). - TransmissionMeta meta = rcvCtx.lastRcv == null ? new TransmissionMeta(rcvCtx.lastSeenErr) : - rcvCtx.lastRcv.state().error(rcvCtx.lastSeenErr); + return; + } - out.writeObject(meta); + if (!newSesId.equals(rcvCtx.sesId)) { + // Attempt to receive file with new session id. Context must be reinited, + // previous session must be failed. + closeRecevier(rcvCtx, nodeId, new IgniteCheckedException("Process has been aborted " + + "by transfer attempt with a new session: " + newSesId)); - // Begin method must be called only once. - if (!rcvCtx.sesStarted) { - rcvCtx.hnd.onBegin(nodeId); + rcvCtx = new ReceiverContext(nodeId, hnd, newSesId); + rcvCtx.active.set(true); - rcvCtx.sesStarted = true; + rcvCtxs.put(topic, rcvCtx); } } - catch (Throwable t) { - rcvCtx.inProgress.set(false); - throw t; - } - finally { - busyLock.readLock().unlock(); - } + // Send previous context state to sync remote and local node (on manager connected). + TransmissionMeta meta = rcvCtx.lastRcv == null ? new TransmissionMeta(rcvCtx.lastSeenErr) : + rcvCtx.lastRcv.state().error(rcvCtx.lastSeenErr); + + out.writeObject(meta); - processOpenedChannel0(topic, rcvCtx, in, out, channel); + receiveFromChannel(topic, rcvCtx, in, out, ch); } catch (Throwable t) { - U.error(log, "The download session cannot be finished due to unexpected error " + - "[ctx=" + rcvCtx + ", sesKey=" + newSesId + ']', t); + U.error(log, "Download session cannot be finished due to an unexpected error [ctx=" + rcvCtx + ']', t); if (rcvCtx != null) { rcvCtx.lastSeenErr = new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t); @@ -2760,24 +2779,34 @@ else if (!rcvCtx.sesId.equals(newSesId)) { } } finally { + if (rcvCtx != null) + rcvCtx.active.set(false); + U.closeQuiet(in); U.closeQuiet(out); - U.closeQuiet(channel); + U.closeQuiet(ch); } } /** * @param topic Topic handler related to. * @param rcvCtx Receiver read context. - * @throws Exception If processing fails. + * @throws IgniteCheckedException If processing fails. */ - private void processOpenedChannel0( + private void receiveFromChannel( Object topic, ReceiverContext rcvCtx, ObjectInputStream in, ObjectOutputStream out, ReadableByteChannel channel - ) throws Exception { + ) throws IgniteCheckedException { + // Begin method must be called only once. + if (!rcvCtx.sesStarted) { + rcvCtx.hnd.onBegin(rcvCtx.nodeId); + + rcvCtx.sesStarted = true; + } + try { while (true) { if (Thread.currentThread().isInterrupted()) @@ -2796,9 +2825,7 @@ private void processOpenedChannel0( break; } - TransmissionMeta meta = new TransmissionMeta(); - - meta.readExternal(in); + TransmissionMeta meta = (TransmissionMeta)in.readObject(); if (rcvCtx.lastRcv == null) { rcvCtx.lastRcv = createReceiver(rcvCtx.nodeId, @@ -2839,8 +2866,8 @@ private void processOpenedChannel0( "Max attempts: " + retryCnt, e); } } - finally { - rcvCtx.inProgress.set(false); + catch (InterruptedException | ClassNotFoundException e) { + throw new IgniteCheckedException(e); } } @@ -2934,19 +2961,19 @@ private static class ReceiverContext { /** The remote node input channel came from. */ private final UUID nodeId; - /** Handler currently in use flag. */ - private final AtomicBoolean inProgress = new AtomicBoolean(); - /** Current sesssion handler. */ @GridToStringExclude private final TransmissionHandler hnd; + /** Unique session request id. */ + private final IgniteUuid sesId; + + /** Handler currently in use flag. */ + private final AtomicBoolean active = new AtomicBoolean(); + /** Flag indicates session started. */ private boolean sesStarted; - /** Unique session request id. */ - private IgniteUuid sesId; - /** The number of retry attempts of current session to wait. */ private int retries; @@ -2962,10 +2989,12 @@ private static class ReceiverContext { /** * @param nodeId Remote node id. * @param hnd Channel handler of current topic. + * @param sesId Unique session request id. */ - public ReceiverContext(UUID nodeId, TransmissionHandler hnd) { + public ReceiverContext(UUID nodeId, TransmissionHandler hnd, IgniteUuid sesId) { this.nodeId = nodeId; this.hnd = hnd; + this.sesId = sesId; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index a07a7076bffc8..2f5c526467a34 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -449,7 +449,6 @@ public void testFileHandlerNextWriterOpened() throws Exception { rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore) { @Override public void onException(UUID nodeId, Throwable err) { assertEquals("Previous session is not closed properly", IgniteCheckedException.class, err.getClass()); - assertTrue(err.getMessage().startsWith("The handler has been aborted")); } @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { @@ -460,6 +459,8 @@ public void testFileHandlerNextWriterOpened() throws Exception { } }); + IgniteCheckedException expectedErr = null; + try (GridIoManager.TransmissionSender sender = snd.context() .io() .openTransmissionSender(rcv.localNode().id(), topic)) { @@ -467,9 +468,11 @@ public void testFileHandlerNextWriterOpened() throws Exception { } catch (IgniteCheckedException e) { // Expected exception. - assertTrue(e.toString(), e.getCause().getMessage().startsWith("Channel processing error")); + expectedErr = e; } + assertNotNull("Transmission must ends with an exception", expectedErr); + //Open next session and complete successfull. try (GridIoManager.TransmissionSender sender = snd.context() .io() @@ -483,8 +486,6 @@ public void testFileHandlerNextWriterOpened() throws Exception { // Remove topic handler and fail rcv.context().io().removeTransmissionHandler(topic); - IgniteCheckedException err = null; - // Open next writer on removed topic. try (GridIoManager.TransmissionSender sender = snd.context() .io() @@ -493,10 +494,10 @@ public void testFileHandlerNextWriterOpened() throws Exception { } catch (IgniteCheckedException e) { // Must catch execption here. - err = e; + expectedErr = e; } - assertNotNull(err); + assertNotNull("Transmission must ends with an exception", expectedErr); } /** From 1cb968d5abc7e023495e2fc604ae8064a0375289 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 4 Aug 2019 00:30:06 +0300 Subject: [PATCH 28/69] IGNITE-10619: create receiver on each channel request --- .../communication/AbstractReceiver.java | 55 ++----------- .../communication/AbstractTransmission.java | 31 ++++--- .../managers/communication/ChunkReceiver.java | 28 +++---- .../managers/communication/FileReceiver.java | 72 ++++++---------- .../managers/communication/FileSender.java | 29 ++++--- .../managers/communication/GridIoManager.java | 82 ++++++++++++------- 6 files changed, 133 insertions(+), 164 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java index e62d71443c4b2..71e2a63d79a22 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java @@ -23,52 +23,33 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; -import static org.apache.ignite.internal.util.IgniteUtils.assertParameter; - /** * Class represents a receiver of data which can be pulled from a channel by chunks of * predefined size. Closes when a transmission of represented object ends. */ abstract class AbstractReceiver extends AbstractTransmission { /** - * @param initMeta Initial file meta info. + * @param meta Initial file meta info. * @param stopChecker Node stop or prcoess interrupt checker. * @param log Ignite logger. * @param chunkSize Size of chunks. */ protected AbstractReceiver( - TransmissionMeta initMeta, + TransmissionMeta meta, BooleanSupplier stopChecker, IgniteLogger log, int chunkSize ) { - super(initMeta, stopChecker, log, chunkSize); + super(meta, stopChecker, log, chunkSize); } /** * @param ch Input channel to read data from. - * @param meta Meta information about receiving file. * @throws IOException If an io exception occurred. * @throws IgniteCheckedException If some check failed. */ - public void receive( - ReadableByteChannel ch, - TransmissionMeta meta - ) throws IOException, IgniteCheckedException { - assert meta != null; - - assertParameter(initMeta.name().equals(meta.name()), "Attempt to load different file " + - "[initMeta=" + initMeta + ", meta=" + meta + ']'); - - assertParameter(initMeta.offset() + transferred == meta.offset(), - "The next chunk offest is incorrect [initMeta=" + initMeta + - ", transferred=" + transferred + ", meta=" + meta + ']'); - - assertParameter(initMeta.count() == meta.count() + transferred, " The count of bytes to transfer for " + - "the next chunk is incorrect [total=" + initMeta.count() + ", transferred=" + transferred + - ", initMeta=" + initMeta + ", meta=" + meta + ']'); - - init(meta); + public void receive(ReadableByteChannel ch) throws IOException, IgniteCheckedException { + init(); // Read data from the input. while (hasNextChunk()) { @@ -80,34 +61,14 @@ public void receive( readChunk(ch); } - assert transferred == initMeta.count() : "The number of transferred bytes are not as expected " + - "[expect=" + initMeta.count() + ", actual=" + transferred + ']'; - } - - /** - * @return Current receiver state written to a {@link TransmissionMeta} instance. - */ - public TransmissionMeta state() { - return new TransmissionMeta(initMeta.name(), - initMeta.offset() + transferred, - initMeta.count() - transferred, - initMeta.params(), - policy(), - null); - } - - /** - * @return Read policy of data handling. - */ - protected TransmissionPolicy policy() { - return initMeta.policy(); + assert transferred == meta.count() : "The number of transferred bytes are not as expected " + + "[expect=" + meta.count() + ", actual=" + transferred + ']'; } /** - * @param meta Meta information about receiving file. * @throws IgniteCheckedException If fails. */ - protected abstract void init(TransmissionMeta meta) throws IgniteCheckedException; + protected abstract void init() throws IgniteCheckedException; /** * @param ch Channel to read data from. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java index cc40057a6d5b7..bd1da546c0cf5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java @@ -39,41 +39,48 @@ abstract class AbstractTransmission implements Closeable { protected final IgniteLogger log; /** Initial meta with file transferred attributes. */ - protected final TransmissionMeta initMeta; + protected TransmissionMeta meta; /** The number of bytes successfully transferred druring iteration. */ protected long transferred; /** - * @param initMeta Initial file meta info. + * @param meta Initial file meta info. * @param stopChecker Node stop or prcoess interrupt checker. * @param log Ignite logger. * @param chunkSize Size of chunks. */ protected AbstractTransmission( - TransmissionMeta initMeta, + TransmissionMeta meta, BooleanSupplier stopChecker, IgniteLogger log, int chunkSize ) { - A.notNull(initMeta, "Initial file meta cannot be null"); - A.notNullOrEmpty(initMeta.name(), "Trasmisson name cannot be empty or null"); - A.ensure(initMeta.offset() >= 0, "File start position cannot be negative"); - A.ensure(initMeta.count() > 0, "Total number of bytes to transfer must be greater than zero"); + A.notNull(meta, "Initial file meta cannot be null"); + A.notNullOrEmpty(meta.name(), "Trasmisson name cannot be empty or null"); + A.ensure(meta.offset() >= 0, "File start position cannot be negative"); + A.ensure(meta.count() > 0, "Total number of bytes to transfer must be greater than zero"); A.notNull(stopChecker, "Process stop checker cannot be null"); A.ensure(chunkSize > 0, "Size of chunks to transfer data must be positive"); this.stopChecker = stopChecker; - this.initMeta = initMeta; this.log = log.getLogger(AbstractTransmission.class); this.chunkSize = chunkSize; + this.meta = meta; } /** - * @return Initial transmission meta. + * @return Current receiver state written to a {@link TransmissionMeta} instance. */ - public TransmissionMeta initMeta() { - return initMeta; + public TransmissionMeta state() { + assert meta != null; + + return new TransmissionMeta(meta.name(), + meta.offset() + transferred, + meta.count() - transferred, + meta.params(), + meta.policy(), + null); } /** @@ -94,7 +101,7 @@ protected boolean stopped() { * @return {@code true} if and only if a chunked object has received all the data it expects. */ protected boolean hasNextChunk() { - return transferred < initMeta.count(); + return transferred < meta.count(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java index 1fb604f908cfd..e222392e4cfe6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java @@ -21,11 +21,11 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.ReadableByteChannel; -import java.util.UUID; import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; +import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -40,34 +40,28 @@ class ChunkReceiver extends AbstractReceiver { private ByteBuffer buf; /** - * @param nodeId The remote node id receive request for transmission from. - * @param initMeta Initial file meta info. + * @param meta Initial file meta info. * @param chunkSize Size of chunks. * @param stopChecker Node stop or prcoess interrupt checker. * @param hnd Transmission handler to process download result. * @param log Ignite looger. - * @throws IgniteCheckedException If fails. */ public ChunkReceiver( - UUID nodeId, - TransmissionMeta initMeta, + TransmissionMeta meta, int chunkSize, BooleanSupplier stopChecker, - TransmissionHandler hnd, + IgniteThrowableConsumer hnd, IgniteLogger log - ) throws IgniteCheckedException { - super(initMeta, stopChecker, log, chunkSize); + ) { + super(meta, stopChecker, log, chunkSize); - assert initMeta.policy() == TransmissionPolicy.CHUNK : initMeta.policy(); + A.notNull(hnd, "ChunkHandler must be provided by transmission handler"); - this.hnd = hnd.chunkHandler(nodeId, initMeta); - - assert this.hnd != null : "ChunkHandler must be provided by transmission handler"; + this.hnd = hnd; } /** {@inheritDoc} */ - @Override protected void init(TransmissionMeta meta) throws IgniteCheckedException { - assert meta != null; + @Override protected void init() throws IgniteCheckedException { assert buf == null; buf = ByteBuffer.allocate(chunkSize); @@ -90,9 +84,9 @@ public ChunkReceiver( // Read will return -1 if remote node close connection. if (res < 0) { - if (transferred + readed != initMeta.count()) { + if (transferred + readed != meta.count()) { throw new IOException("Input data channel reached its end, but file has not fully loaded " + - "[transferred=" + transferred + ", readed=" + readed + ", total=" + initMeta.count() + ']'); + "[transferred=" + transferred + ", readed=" + readed + ", total=" + meta.count() + ']'); } break; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index 906a31962cd51..a177dcc749195 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -22,19 +22,18 @@ import java.nio.channels.FileChannel; import java.nio.channels.ReadableByteChannel; import java.nio.file.Files; -import java.util.UUID; import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.util.lang.IgniteOutClosureX; import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import static org.apache.ignite.internal.util.IgniteUtils.assertParameter; - /** * Class represents a chunk data receiver which is pulling data from channel vi * {@link FileChannel#transferFrom(ReadableByteChannel, long, long)}. @@ -42,10 +41,10 @@ class FileReceiver extends AbstractReceiver { /** The default factory to provide IO oprations over underlying file. */ @GridToStringExclude - private final FileIOFactory fileIoFactory; + private final FileIOFactory factory; /** Handler to notify when a file has been processed. */ - private final IgniteThrowableConsumer hnd; + private final IgniteOutClosureX> hndProvider; /** The abstract java representation of the chunked file. */ private File file; @@ -55,65 +54,48 @@ class FileReceiver extends AbstractReceiver { private FileIO fileIo; /** - * @param nodeId The remote node id receive request for transmission from. - * @param initMeta Initial file meta info. + * @param meta Initial file meta info. * @param stopChecker Node stop or prcoess interrupt checker. * @param factory Factory to produce IO interface on files. - * @param hnd Transmission handler to process download result. + * @param hndProvider Transmission handler provider to process download result. * @param log Ignite logger. - * @throws IgniteCheckedException If fails. */ public FileReceiver( - UUID nodeId, - TransmissionMeta initMeta, + TransmissionMeta meta, int chunkSize, BooleanSupplier stopChecker, FileIOFactory factory, - TransmissionHandler hnd, + IgniteOutClosureX> hndProvider, + String fileAbsPath, IgniteLogger log - ) throws IgniteCheckedException { - super(initMeta, stopChecker, log, chunkSize); - - assert initMeta.policy() == TransmissionPolicy.FILE : initMeta.policy(); - - fileIoFactory = factory; - this.hnd = hnd.fileHandler(nodeId, initMeta); - - assert this.hnd != null : "FileHandler must be provided by transmission handler"; + ) { + super(meta, stopChecker, log, chunkSize); - String fileAbsPath = hnd.filePath(nodeId, initMeta); - - if (fileAbsPath == null || fileAbsPath.trim().isEmpty()) - throw new IgniteCheckedException("File receiver absolute path cannot be empty or null. Receiver cannot be" + - " initialized: " + this); + A.notNull(hndProvider, "FileHandler must be provided by transmission handler"); + A.notNull(fileAbsPath, "File absolute path cannot be null"); + A.ensure(!fileAbsPath.trim().isEmpty(), "File absolute path cannot be empty "); + this.factory = factory; + this.hndProvider = hndProvider; file = new File(fileAbsPath); } /** {@inheritDoc} */ - @Override public void receive( - ReadableByteChannel ch, - TransmissionMeta meta - ) throws IOException, IgniteCheckedException { - super.receive(ch, meta); - - if (transferred == initMeta.count()) - hnd.accept(file); + @Override public void receive(ReadableByteChannel ch) throws IOException, IgniteCheckedException { + super.receive(ch); + + if (transferred == meta.count()) + hndProvider.apply().accept(file); } /** {@inheritDoc} */ - @Override protected void init(TransmissionMeta meta) throws IgniteCheckedException { - assert meta != null; + @Override protected void init() throws IgniteCheckedException { assert fileIo == null; - assertParameter(meta.name().equals(initMeta.name()), "Read operation stopped. " + - "Attempt to receive a new file from channel, while the previous was not fully loaded " + - "[meta=" + meta + ", prevFile=" + initMeta.name() + ']'); - try { - fileIo = fileIoFactory.create(file); + fileIo = factory.create(file); - fileIo.position(initMeta.offset() + transferred); + fileIo.position(meta.offset() + transferred); } catch (IOException e) { throw new IgniteCheckedException("Unable to open destination file. Receiver will will be stopped", e); @@ -122,9 +104,9 @@ public FileReceiver( /** {@inheritDoc} */ @Override protected void readChunk(ReadableByteChannel ch) throws IOException { - long batchSize = Math.min(chunkSize, initMeta.count() - transferred); + long batchSize = Math.min(chunkSize, meta.count() - transferred); - long readed = fileIo.transferFrom(ch, initMeta.offset() + transferred, batchSize); + long readed = fileIo.transferFrom(ch, meta.offset() + transferred, batchSize); if (readed > 0) transferred += readed; @@ -137,7 +119,7 @@ public FileReceiver( fileIo = null; try { - if (transferred != initMeta.count()) + if (stopped() && transferred != meta.count()) Files.delete(file.toPath()); } catch (IOException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index cdc53a37b69cb..2cf9057ca33a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -63,6 +63,7 @@ class FileSender extends AbstractTransmission { * @param off File offset. * @param cnt Number of bytes to transfer. * @param params Additional file params. + * @param plc Policy of handling data on remote. * @param stopChecker Node stop or prcoess interrupt checker. * @param log Ignite logger. * @param factory Factory to produce IO interface on given file. @@ -73,15 +74,13 @@ public FileSender( long off, long cnt, Map params, + TransmissionPolicy plc, BooleanSupplier stopChecker, IgniteLogger log, FileIOFactory factory, int chunkSize ) { - super(new TransmissionMeta(file.getName(), off, cnt, params, null, null), - stopChecker, - log, - chunkSize); + super(new TransmissionMeta(file.getName(), off, cnt, params, plc, null), stopChecker, log, chunkSize); assert file != null; @@ -120,10 +119,10 @@ public void send(WritableByteChannel ch, oo.writeBoolean(false); // Send meta about current file to remote. - oo.writeObject(new TransmissionMeta(initMeta.name(), - initMeta.offset() + transferred, - initMeta.count() - transferred, - initMeta.params(), + oo.writeObject(new TransmissionMeta(meta.name(), + meta.offset() + transferred, + meta.count() - transferred, + meta.params(), plc, null)); @@ -136,7 +135,7 @@ public void send(WritableByteChannel ch, writeChunk(ch); } - assert transferred == initMeta.count() : "File is not fully transferred [expect=" + initMeta.count() + + assert transferred == meta.count() : "File is not fully transferred [expect=" + meta.count() + ", actual=" + transferred + ']'; } @@ -151,13 +150,13 @@ private void state(TransmissionMeta connMeta) { if (connMeta.offset() < 0) return; - long uploadedBytes = connMeta.offset() - initMeta.offset(); + long uploadedBytes = connMeta.offset() - meta.offset(); - assertParameter(initMeta.name().equals(connMeta.name()), "Attempt to transfer different file " + - "while previous is not completed [initMeta=" + initMeta + ", meta=" + connMeta + ']'); + assertParameter(meta.name().equals(connMeta.name()), "Attempt to transfer different file " + + "while previous is not completed [meta=" + meta + ", meta=" + connMeta + ']'); assertParameter(uploadedBytes >= 0, "Incorrect sync meta [offset=" + connMeta.offset() + - ", initMeta=" + initMeta + ']'); + ", meta=" + meta + ']'); // No need to set new file position, if it is not changed. if (uploadedBytes == 0) @@ -173,9 +172,9 @@ private void state(TransmissionMeta connMeta) { * @throws IOException If fails. */ private void writeChunk(WritableByteChannel ch) throws IOException { - long batchSize = Math.min(chunkSize, initMeta.count() - transferred); + long batchSize = Math.min(chunkSize, meta.count() - transferred); - long sent = fileIo.transferTo(initMeta.offset() + transferred, batchSize, ch); + long sent = fileIo.transferTo(meta.offset() + transferred, batchSize, ch); if (sent > 0) transferred += sent; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 38bb5acb1dc31..650d71aba192f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -105,7 +105,9 @@ import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridTuple3; +import org.apache.ignite.internal.util.lang.IgniteOutClosureX; import org.apache.ignite.internal.util.lang.IgnitePair; +import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; @@ -2670,8 +2672,6 @@ private static void closeRecevier(ReceiverContext ctx, UUID nodeId, IgniteChecke ctx.interrupted = true; - U.closeQuiet(ctx.lastRcv); - ctx.hnd.onException(nodeId, ex); } @@ -2762,10 +2762,7 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { } // Send previous context state to sync remote and local node (on manager connected). - TransmissionMeta meta = rcvCtx.lastRcv == null ? new TransmissionMeta(rcvCtx.lastSeenErr) : - rcvCtx.lastRcv.state().error(rcvCtx.lastSeenErr); - - out.writeObject(meta); + out.writeObject(rcvCtx.lastState); receiveFromChannel(topic, rcvCtx, in, out, ch); } @@ -2773,7 +2770,13 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { U.error(log, "Download session cannot be finished due to an unexpected error [ctx=" + rcvCtx + ']', t); if (rcvCtx != null) { - rcvCtx.lastSeenErr = new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t); + IgniteCheckedException ex = new IgniteCheckedException("Channel processing error " + + "[nodeId=" + nodeId + ']', t); + + if (rcvCtx.lastRcv == null) + rcvCtx.lastState.error(ex); + else + rcvCtx.lastState = rcvCtx.lastRcv.state().error(ex); rcvCtx.hnd.onException(nodeId, t); } @@ -2798,7 +2801,7 @@ private void receiveFromChannel( ReceiverContext rcvCtx, ObjectInputStream in, ObjectOutputStream out, - ReadableByteChannel channel + ReadableByteChannel ch ) throws IgniteCheckedException { // Begin method must be called only once. if (!rcvCtx.sesStarted) { @@ -2827,20 +2830,21 @@ private void receiveFromChannel( TransmissionMeta meta = (TransmissionMeta)in.readObject(); - if (rcvCtx.lastRcv == null) { - rcvCtx.lastRcv = createReceiver(rcvCtx.nodeId, - rcvCtx.hnd, - meta, - () -> stopping || rcvCtx.interrupted); - } + if (rcvCtx.lastRcv != null) + validate(rcvCtx.lastRcv.state(), meta); + + rcvCtx.lastRcv = createReceiver(rcvCtx.nodeId, + rcvCtx.hnd, + meta, + () -> stopping || rcvCtx.interrupted); try (AbstractReceiver rcv = rcvCtx.lastRcv) { long startTime = U.currentTimeMillis(); - rcv.receive(channel, meta); + rcv.receive(ch); // Write processing ack. - out.writeLong(rcv.transferred()); + out.writeBoolean(true); out.flush(); rcvCtx.lastRcv = null; @@ -2848,7 +2852,7 @@ private void receiveFromChannel( long downloadTime = U.currentTimeMillis() - startTime; U.log(log, "File has been received " + - "[name=" + rcv.initMeta().name() + ", transferred=" + rcv.transferred() + + "[name=" + rcv.state().name() + ", transferred=" + rcv.transferred() + ", time=" + (double)((downloadTime) / 1000) + " sec" + ", retries=" + rcvCtx.retries + ", remoteId=" + rcvCtx.nodeId + ']'); } @@ -2871,6 +2875,25 @@ private void receiveFromChannel( } } + /** + * @param prev Previous available transmission meta. + * @param next Next transmission meta. + * @throws IgniteCheckedException If fails. + */ + private void validate(TransmissionMeta prev, TransmissionMeta next) throws IgniteCheckedException { + assertParameter(prev.name().equals(next.name()), "Attempt to load different file " + + "[prev=" + prev + ", next=" + next + ']'); + + assertParameter(prev.offset() == next.offset(), + "The next chunk offest is incorrect [prev=" + prev + ", meta=" + next + ']'); + + assertParameter(prev.count() == next.count(), " The count of bytes to transfer for " + + "the next chunk is incorrect [prev=" + prev + ", next=" + next + ']'); + + assertParameter(prev.policy() == next.policy(), "Attemt to continue file upload with" + + " different transmission policy [prev=" + prev + ", next=" + next + ']'); + } + /** * Set factory to produce an FileIO abstraction over sended\received files. * @see #fileIoFactory @@ -2898,23 +2921,26 @@ private AbstractReceiver createReceiver( switch (meta.policy()) { case FILE: return new FileReceiver( - nodeId, meta, chunkSize, stopChecker, fileIoFactory, - hnd, + new IgniteOutClosureX>() { + @Override public IgniteThrowableConsumer applyx() throws IgniteCheckedException { + return hnd.fileHandler(nodeId, meta); + } + }, + hnd.filePath(nodeId, meta), log); case CHUNK: return new ChunkReceiver( - nodeId, meta, ctx.config() .getDataStorageConfiguration() .getPageSize(), stopChecker, - hnd, + hnd.chunkHandler(nodeId, meta), log); default: @@ -2980,12 +3006,12 @@ private static class ReceiverContext { /** Last infinished downloading object. */ private AbstractReceiver lastRcv; - /** Last error occurred while channel is processed by registered session handler. */ - private IgniteCheckedException lastSeenErr; - /** Flag indicates that current file handling process must be interrupted. */ private volatile boolean interrupted; + /** Last saved state about file data processing. */ + private TransmissionMeta lastState = new TransmissionMeta(); + /** * @param nodeId Remote node id. * @param hnd Channel handler of current topic. @@ -3175,6 +3201,7 @@ public void send( offset, cnt, params, + plc, () -> stopping || senderStopFlags.get(sesKey).get(), log, fileIoFactory, @@ -3211,10 +3238,9 @@ public void send( snd.send(channel, out, connMeta, plc); // Read file received acknowledge. - long total = in.readLong(); + boolean written = in.readBoolean(); - assert total == snd.transferred() : "File is not fully written [expect=" + total + - ", transferred=" + snd.transferred() + ']'; + assert written : "File is not fully written :" + file.getAbsolutePath(); break; } @@ -3226,7 +3252,7 @@ public void send( "will be re-establishing [remoteId=" + remoteId + ", file=" + file.getName() + ", sesKey=" + sesKey + ", retries=" + retries + ", transferred=" + snd.transferred() + - ", total=" + snd.initMeta().count() + ']', e); + ", total=" + snd.state().count() + ']', e); retries++; From 7a66c1ead935323010e43d599e1a609e3cd1a8dc Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 4 Aug 2019 01:00:06 +0300 Subject: [PATCH 29/69] IGNITE-10619: last receiver, last meta state code simplification --- .../managers/communication/GridIoManager.java | 34 +++++++++++-------- 1 file changed, 20 insertions(+), 14 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 650d71aba192f..8bd2e11c82994 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2762,7 +2762,7 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { } // Send previous context state to sync remote and local node (on manager connected). - out.writeObject(rcvCtx.lastState); + out.writeObject(rcvCtx.lastState == null ? new TransmissionMeta() : rcvCtx.lastState); receiveFromChannel(topic, rcvCtx, in, out, ch); } @@ -2773,10 +2773,8 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { IgniteCheckedException ex = new IgniteCheckedException("Channel processing error " + "[nodeId=" + nodeId + ']', t); - if (rcvCtx.lastRcv == null) - rcvCtx.lastState.error(ex); - else - rcvCtx.lastState = rcvCtx.lastRcv.state().error(ex); + rcvCtx.lastState = rcvCtx.lastState == null ? + new TransmissionMeta(ex) : rcvCtx.lastState.error(ex); rcvCtx.hnd.onException(nodeId, t); } @@ -2830,15 +2828,18 @@ private void receiveFromChannel( TransmissionMeta meta = (TransmissionMeta)in.readObject(); - if (rcvCtx.lastRcv != null) - validate(rcvCtx.lastRcv.state(), meta); + if (rcvCtx.lastState == null) + rcvCtx.lastState = meta; + + validate(rcvCtx.lastState, meta); - rcvCtx.lastRcv = createReceiver(rcvCtx.nodeId, + try (AbstractReceiver rcv = createReceiver(rcvCtx.nodeId, rcvCtx.hnd, meta, - () -> stopping || rcvCtx.interrupted); + () -> stopping || rcvCtx.interrupted) + ) { + rcvCtx.rcv = rcv; - try (AbstractReceiver rcv = rcvCtx.lastRcv) { long startTime = U.currentTimeMillis(); rcv.receive(ch); @@ -2847,14 +2848,19 @@ private void receiveFromChannel( out.writeBoolean(true); out.flush(); - rcvCtx.lastRcv = null; - long downloadTime = U.currentTimeMillis() - startTime; U.log(log, "File has been received " + "[name=" + rcv.state().name() + ", transferred=" + rcv.transferred() + ", time=" + (double)((downloadTime) / 1000) + " sec" + ", retries=" + rcvCtx.retries + ", remoteId=" + rcvCtx.nodeId + ']'); + + rcvCtx.lastState = null; + } + catch (Throwable e) { + rcvCtx.lastState = rcvCtx.rcv.state(); + + throw e; } } } @@ -3004,13 +3010,13 @@ private static class ReceiverContext { private int retries; /** Last infinished downloading object. */ - private AbstractReceiver lastRcv; + private AbstractReceiver rcv; /** Flag indicates that current file handling process must be interrupted. */ private volatile boolean interrupted; /** Last saved state about file data processing. */ - private TransmissionMeta lastState = new TransmissionMeta(); + private TransmissionMeta lastState; /** * @param nodeId Remote node id. From e3bc08346bd3b1aa49da3cbb23ce8430315c6c8b Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 4 Aug 2019 01:28:27 +0300 Subject: [PATCH 30/69] IGNITE-10619: add cleanup method for unfinished resources --- .../communication/AbstractReceiver.java | 5 ++++ .../managers/communication/ChunkReceiver.java | 5 ++++ .../managers/communication/FileReceiver.java | 26 ++++++++++++++----- .../managers/communication/GridIoManager.java | 3 +++ 4 files changed, 32 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java index 71e2a63d79a22..1f9160b8967e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java @@ -65,6 +65,11 @@ public void receive(ReadableByteChannel ch) throws IOException, IgniteCheckedExc "[expect=" + meta.count() + ", actual=" + transferred + ']'; } + /** + * Cleanup unused or unfinished receivers resources. + */ + public abstract void cleanup(); + /** * @throws IgniteCheckedException If fails. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java index e222392e4cfe6..a7090d221bb06 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java @@ -108,6 +108,11 @@ public ChunkReceiver( hnd.accept(buf); } + /** {@inheritDoc} */ + @Override public void cleanup() { + // Nothing to clean. + } + /** {@inheritDoc} */ @Override public void close() throws IOException { buf = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index a177dcc749195..32bef6d9a0eb5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -27,6 +27,7 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.IgniteOutClosureX; import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -39,6 +40,9 @@ * {@link FileChannel#transferFrom(ReadableByteChannel, long, long)}. */ class FileReceiver extends AbstractReceiver { + /** Future will be done when receiver is closed. */ + protected final GridFutureAdapter closeFut = new GridFutureAdapter<>(); + /** The default factory to provide IO oprations over underlying file. */ @GridToStringExclude private final FileIOFactory factory; @@ -91,6 +95,7 @@ public FileReceiver( /** {@inheritDoc} */ @Override protected void init() throws IgniteCheckedException { assert fileIo == null; + assert !closeFut.isDone(); try { fileIo = factory.create(file); @@ -113,20 +118,27 @@ public FileReceiver( } /** {@inheritDoc} */ - @Override public void close() throws IOException { - U.closeQuiet(fileIo); - - fileIo = null; - + @Override public void cleanup() { try { - if (stopped() && transferred != meta.count()) + closeFut.get(); + + if (transferred != meta.count()) Files.delete(file.toPath()); } - catch (IOException e) { + catch (IOException | IgniteCheckedException e) { U.error(log, "Error deleting not fully loaded file: " + file, e); } } + /** {@inheritDoc} */ + @Override public void close() throws IOException { + U.closeQuiet(fileIo); + + fileIo = null; + + closeFut.onDone(); + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(FileReceiver.class, this, "super", super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 8bd2e11c82994..3e99b30dda50f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2672,6 +2672,9 @@ private static void closeRecevier(ReceiverContext ctx, UUID nodeId, IgniteChecke ctx.interrupted = true; + if (ctx.rcv != null) + ctx.rcv.cleanup(); + ctx.hnd.onException(nodeId, ex); } From 39484b450131c205739c10962f06f95999c771c0 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 13:13:39 +0300 Subject: [PATCH 31/69] IGNITE-10619: rename sender internal local variables --- .../managers/communication/FileSender.java | 22 +++++++++---------- .../managers/communication/GridIoManager.java | 12 +++++----- 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 2cf9057ca33a7..08484a24beb41 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -91,14 +91,14 @@ public FileSender( /** * @param ch Output channel to write file to. * @param oo Channel to write meta info to. - * @param connMeta Connection meta received. + * @param rcvMeta Connection meta received. * @param plc Policy of how data will be handled on remote node. * @throws IOException If a transport exception occurred. * @throws IgniteCheckedException If fails. */ public void send(WritableByteChannel ch, ObjectOutput oo, - @Nullable TransmissionMeta connMeta, + @Nullable TransmissionMeta rcvMeta, TransmissionPolicy plc ) throws IOException, IgniteCheckedException { try { @@ -112,8 +112,7 @@ public void send(WritableByteChannel ch, } // If not the initial connection for the current session. - if (connMeta != null) - state(connMeta); + updateSenderState(rcvMeta); // Write flag to remote to keep currnet transmission opened. oo.writeBoolean(false); @@ -140,22 +139,21 @@ public void send(WritableByteChannel ch, } /** - * @param connMeta Conneciton meta info. + * @param rcvMeta Conneciton meta info. */ - private void state(TransmissionMeta connMeta) { - assert connMeta != null; + private void updateSenderState(TransmissionMeta rcvMeta) { assert fileIo != null; // The remote node doesn't have a file meta info. - if (connMeta.offset() < 0) + if (rcvMeta == null || rcvMeta.offset() < 0) return; - long uploadedBytes = connMeta.offset() - meta.offset(); + long uploadedBytes = rcvMeta.offset() - meta.offset(); - assertParameter(meta.name().equals(connMeta.name()), "Attempt to transfer different file " + - "while previous is not completed [meta=" + meta + ", meta=" + connMeta + ']'); + assertParameter(meta.name().equals(rcvMeta.name()), "Attempt to transfer different file " + + "while previous is not completed [meta=" + meta + ", meta=" + rcvMeta + ']'); - assertParameter(uploadedBytes >= 0, "Incorrect sync meta [offset=" + connMeta.offset() + + assertParameter(uploadedBytes >= 0, "Incorrect sync meta [offset=" + rcvMeta.offset() + ", meta=" + meta + ']'); // No need to set new file position, if it is not changed. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 3e99b30dda50f..b7c5bb7c58ae8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -3232,19 +3232,19 @@ public void send( throw new NodeStoppingException("Operation has been cancelled (node is stopping)"); try { - TransmissionMeta connMeta = null; + TransmissionMeta rcvMeta = null; if (out == null && in == null) { - connMeta = connect(); + rcvMeta = connect(); - assert connMeta != null; + assert rcvMeta != null; // Stop in case of any error occurred on remote node during file processing. - if (connMeta.error() != null) - throw connMeta.error(); + if (rcvMeta.error() != null) + throw rcvMeta.error(); } - snd.send(channel, out, connMeta, plc); + snd.send(channel, out, rcvMeta, plc); // Read file received acknowledge. boolean written = in.readBoolean(); From 33e68760f967c0f852e11c77c2579526e1112eab Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 13:17:35 +0300 Subject: [PATCH 32/69] IGNITE-10619: remove unused plc variable --- .../ignite/internal/managers/communication/FileSender.java | 6 ++---- .../internal/managers/communication/GridIoManager.java | 2 +- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 08484a24beb41..98be57c1e1e48 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -92,14 +92,12 @@ public FileSender( * @param ch Output channel to write file to. * @param oo Channel to write meta info to. * @param rcvMeta Connection meta received. - * @param plc Policy of how data will be handled on remote node. * @throws IOException If a transport exception occurred. * @throws IgniteCheckedException If fails. */ public void send(WritableByteChannel ch, ObjectOutput oo, - @Nullable TransmissionMeta rcvMeta, - TransmissionPolicy plc + @Nullable TransmissionMeta rcvMeta ) throws IOException, IgniteCheckedException { try { // Can be not null if reconnection is going to be occurred. @@ -122,7 +120,7 @@ public void send(WritableByteChannel ch, meta.offset() + transferred, meta.count() - transferred, meta.params(), - plc, + meta.policy(), null)); while (hasNextChunk()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index b7c5bb7c58ae8..4910a553cf0a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -3244,7 +3244,7 @@ public void send( throw rcvMeta.error(); } - snd.send(channel, out, rcvMeta, plc); + snd.send(channel, out, rcvMeta); // Read file received acknowledge. boolean written = in.readBoolean(); From c66b751fff796a8169a9817cbef3bb5087351363 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 13:25:10 +0300 Subject: [PATCH 33/69] IGNITE-10619: minor code changes --- .../ignite/internal/managers/communication/GridIoManager.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 4910a553cf0a8..07cf8c758dd7e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -3234,10 +3234,11 @@ public void send( try { TransmissionMeta rcvMeta = null; + // in/out streams are not null if file has been sent successfully if (out == null && in == null) { rcvMeta = connect(); - assert rcvMeta != null; + assert rcvMeta != null : "Remote recevier has not sent its meta"; // Stop in case of any error occurred on remote node during file processing. if (rcvMeta.error() != null) From c777b9c6b979bd3771646eabf308bd3a6010a5cd Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 19:05:14 +0300 Subject: [PATCH 34/69] IGNITE-10619: add timeout object, close receiver only when it is need --- .../communication/AbstractReceiver.java | 12 --- .../managers/communication/ChunkReceiver.java | 12 +-- .../managers/communication/FileReceiver.java | 56 +++++------ .../managers/communication/FileSender.java | 23 ++--- .../managers/communication/GridIoManager.java | 92 ++++++++++++------- 5 files changed, 95 insertions(+), 100 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java index 1f9160b8967e4..84c2879c7c234 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java @@ -49,8 +49,6 @@ protected AbstractReceiver( * @throws IgniteCheckedException If some check failed. */ public void receive(ReadableByteChannel ch) throws IOException, IgniteCheckedException { - init(); - // Read data from the input. while (hasNextChunk()) { if (Thread.currentThread().isInterrupted() || stopped()) { @@ -65,16 +63,6 @@ public void receive(ReadableByteChannel ch) throws IOException, IgniteCheckedExc "[expect=" + meta.count() + ", actual=" + transferred + ']'; } - /** - * Cleanup unused or unfinished receivers resources. - */ - public abstract void cleanup(); - - /** - * @throws IgniteCheckedException If fails. - */ - protected abstract void init() throws IgniteCheckedException; - /** * @param ch Channel to read data from. * @throws IOException If fails. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java index a7090d221bb06..a134a7570f26f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java @@ -58,11 +58,6 @@ public ChunkReceiver( A.notNull(hnd, "ChunkHandler must be provided by transmission handler"); this.hnd = hnd; - } - - /** {@inheritDoc} */ - @Override protected void init() throws IgniteCheckedException { - assert buf == null; buf = ByteBuffer.allocate(chunkSize); buf.order(ByteOrder.nativeOrder()); @@ -109,12 +104,7 @@ public ChunkReceiver( } /** {@inheritDoc} */ - @Override public void cleanup() { - // Nothing to clean. - } - - /** {@inheritDoc} */ - @Override public void close() throws IOException { + @Override public void close() { buf = null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index 32bef6d9a0eb5..b4f44e4e65026 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -62,7 +62,9 @@ class FileReceiver extends AbstractReceiver { * @param stopChecker Node stop or prcoess interrupt checker. * @param factory Factory to produce IO interface on files. * @param hndProvider Transmission handler provider to process download result. + * @param path File path to destination receiver source. * @param log Ignite logger. + * @throws IgniteCheckedException If fails. */ public FileReceiver( TransmissionMeta meta, @@ -70,45 +72,42 @@ public FileReceiver( BooleanSupplier stopChecker, FileIOFactory factory, IgniteOutClosureX> hndProvider, - String fileAbsPath, + String path, IgniteLogger log - ) { + ) throws IgniteCheckedException { super(meta, stopChecker, log, chunkSize); A.notNull(hndProvider, "FileHandler must be provided by transmission handler"); - A.notNull(fileAbsPath, "File absolute path cannot be null"); - A.ensure(!fileAbsPath.trim().isEmpty(), "File absolute path cannot be empty "); + A.notNull(path, "File absolute path cannot be null"); + A.ensure(!path.trim().isEmpty(), "File absolute path cannot be empty "); this.factory = factory; this.hndProvider = hndProvider; - file = new File(fileAbsPath); - } - - /** {@inheritDoc} */ - @Override public void receive(ReadableByteChannel ch) throws IOException, IgniteCheckedException { - super.receive(ch); - - if (transferred == meta.count()) - hndProvider.apply().accept(file); - } - /** {@inheritDoc} */ - @Override protected void init() throws IgniteCheckedException { - assert fileIo == null; - assert !closeFut.isDone(); + file = new File(path); try { fileIo = factory.create(file); - fileIo.position(meta.offset() + transferred); + fileIo.position(meta.offset()); } catch (IOException e) { throw new IgniteCheckedException("Unable to open destination file. Receiver will will be stopped", e); } } + /** {@inheritDoc} */ + @Override public void receive(ReadableByteChannel ch) throws IOException, IgniteCheckedException { + super.receive(ch); + + if (transferred == meta.count()) + hndProvider.apply().accept(file); + } + /** {@inheritDoc} */ @Override protected void readChunk(ReadableByteChannel ch) throws IOException { + assert fileIo != null; + long batchSize = Math.min(chunkSize, meta.count() - transferred); long readed = fileIo.transferFrom(ch, meta.offset() + transferred, batchSize); @@ -118,27 +117,20 @@ public FileReceiver( } /** {@inheritDoc} */ - @Override public void cleanup() { - try { - closeFut.get(); + @Override public void close() { + U.closeQuiet(fileIo); + + fileIo = null; + try { if (transferred != meta.count()) Files.delete(file.toPath()); } - catch (IOException | IgniteCheckedException e) { + catch (IOException e) { U.error(log, "Error deleting not fully loaded file: " + file, e); } } - /** {@inheritDoc} */ - @Override public void close() throws IOException { - U.closeQuiet(fileIo); - - fileIo = null; - - closeFut.onDone(); - } - /** {@inheritDoc} */ @Override public String toString() { return S.toString(FileReceiver.class, this, "super", super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 98be57c1e1e48..0fa9e21ce246e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -68,6 +68,7 @@ class FileSender extends AbstractTransmission { * @param log Ignite logger. * @param factory Factory to produce IO interface on given file. * @param chunkSize Size of chunks. + * @throws IgniteCheckedException If fails. */ public FileSender( File file, @@ -79,13 +80,23 @@ public FileSender( IgniteLogger log, FileIOFactory factory, int chunkSize - ) { + ) throws IgniteCheckedException { super(new TransmissionMeta(file.getName(), off, cnt, params, plc, null), stopChecker, log, chunkSize); assert file != null; this.file = file; this.factory = factory; + + try { + // Can be not null if reconnection is going to be occurred. + if (fileIo == null) + fileIo = factory.create(file); + } + catch (IOException e) { + // Consider this IO exeption as a user one (not the network exception) and interrupt upload process. + throw new IgniteCheckedException("Unable to initialize source file. File sender upload will be stopped", e); + } } /** @@ -99,16 +110,6 @@ public void send(WritableByteChannel ch, ObjectOutput oo, @Nullable TransmissionMeta rcvMeta ) throws IOException, IgniteCheckedException { - try { - // Can be not null if reconnection is going to be occurred. - if (fileIo == null) - fileIo = factory.create(file); - } - catch (IOException e) { - // Consider this IO exeption as a user one (not the network exception) and interrupt upload process. - throw new IgniteCheckedException("Unable to initialize source file. File sender upload will be stopped", e); - } - // If not the initial connection for the current session. updateSenderState(rcvMeta); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 07cf8c758dd7e..739ba2cf3339c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -934,7 +934,6 @@ private void format(StringBuilder b, Collection> pairs, SimpleD it.remove(); closeRecevier(e.getValue(), - nodeId, new ClusterTopologyCheckedException("Remove node left the grid. " + "Receiver has been stopped : " + nodeId)); } @@ -1106,6 +1105,13 @@ private void format(StringBuilder b, Collection> pairs, SimpleD stopping = true; topicTransmissionHnds.clear(); + + for (ReceiverContext rctx : rcvCtxs.values()) { + closeRecevier(rctx, new NodeStoppingException("Local node io manager requested to be stopped: " + + ctx.localNodeId())); + } + + rcvCtxs.clear(); } finally { busyLock.writeLock().unlock(); @@ -1873,8 +1879,8 @@ public void removeTransmissionHandler(Object topic) { topicTransmissionHnds.remove(topic); closeRecevier(rcvCtxs.remove(topic), - ctx.localNodeId(), - new IgniteCheckedException("Receiver has been closed due to removing corresponding transmission handler")); + new IgniteCheckedException("Receiver has been closed due to removing corresponding transmission handler " + + "on local node [nodeId=" + ctx.localNodeId() + ']')); } /** @@ -2663,19 +2669,20 @@ public int getOutboundMessagesQueueSize() { /** * @param ctx Receiver context to use. - * @param nodeId Node id caused receiver to close. * @param ex Exception to close receiver with. */ - private static void closeRecevier(ReceiverContext ctx, UUID nodeId, IgniteCheckedException ex) { + private static void closeRecevier(ReceiverContext ctx, IgniteCheckedException ex) { if (ctx == null) return; - ctx.interrupted = true; + if (ctx.interrupted.compareAndSet(false, true)) { + U.closeQuiet(ctx.rcv); - if (ctx.rcv != null) - ctx.rcv.cleanup(); + ctx.lastState = ctx.lastState == null ? + new TransmissionMeta(ex) : ctx.lastState.error(ex); - ctx.hnd.onException(nodeId, ex); + ctx.hnd.onException(ctx.nodeId, ex); + } } /** @@ -2726,6 +2733,9 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa synchronized (rcvCtxs) { rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(nodeId, hnd, newSesId)); + if (rcvCtx.timeoutObj != null) + ctx.timeout().removeTimeoutObject(rcvCtx.timeoutObj); + // Do not allow multiple connection for the same session boolean activated = rcvCtx.active.compareAndSet(false, true); @@ -2754,8 +2764,8 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { if (!newSesId.equals(rcvCtx.sesId)) { // Attempt to receive file with new session id. Context must be reinited, // previous session must be failed. - closeRecevier(rcvCtx, nodeId, new IgniteCheckedException("Process has been aborted " + - "by transfer attempt with a new session: " + newSesId)); + closeRecevier(rcvCtx, new IgniteCheckedException("Process has been aborted " + + "by transfer attempt with a new session [sesId=" + newSesId + ", nodeId=" + nodeId + ']')); rcvCtx = new ReceiverContext(nodeId, hnd, newSesId); rcvCtx.active.set(true); @@ -2772,15 +2782,7 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { catch (Throwable t) { U.error(log, "Download session cannot be finished due to an unexpected error [ctx=" + rcvCtx + ']', t); - if (rcvCtx != null) { - IgniteCheckedException ex = new IgniteCheckedException("Channel processing error " + - "[nodeId=" + nodeId + ']', t); - - rcvCtx.lastState = rcvCtx.lastState == null ? - new TransmissionMeta(ex) : rcvCtx.lastState.error(ex); - - rcvCtx.hnd.onException(nodeId, t); - } + closeRecevier(rcvCtx, new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t)); } finally { if (rcvCtx != null) @@ -2831,34 +2833,36 @@ private void receiveFromChannel( TransmissionMeta meta = (TransmissionMeta)in.readObject(); - if (rcvCtx.lastState == null) + if (rcvCtx.rcv == null) { + rcvCtx.rcv = createReceiver(rcvCtx.nodeId, + rcvCtx.hnd, + meta, + () -> stopping || rcvCtx.interrupted.get()); + rcvCtx.lastState = meta; + } validate(rcvCtx.lastState, meta); - try (AbstractReceiver rcv = createReceiver(rcvCtx.nodeId, - rcvCtx.hnd, - meta, - () -> stopping || rcvCtx.interrupted) - ) { - rcvCtx.rcv = rcv; - + try { long startTime = U.currentTimeMillis(); - rcv.receive(ch); + rcvCtx.rcv.receive(ch); // Write processing ack. out.writeBoolean(true); out.flush(); + rcvCtx.rcv.close(); + long downloadTime = U.currentTimeMillis() - startTime; U.log(log, "File has been received " + - "[name=" + rcv.state().name() + ", transferred=" + rcv.transferred() + + "[name=" + rcvCtx.rcv.state().name() + ", transferred=" + rcvCtx.rcv.transferred() + ", time=" + (double)((downloadTime) / 1000) + " sec" + ", retries=" + rcvCtx.retries + ", remoteId=" + rcvCtx.nodeId + ']'); - rcvCtx.lastState = null; + rcvCtx.rcv = null; } catch (Throwable e) { rcvCtx.lastState = rcvCtx.rcv.state(); @@ -2878,6 +2882,23 @@ private void receiveFromChannel( throw new IgniteCheckedException("Number of retry attempts to download file exceeded the limit. " + "Max attempts: " + retryCnt, e); } + + ctx.timeout().addTimeoutObject(rcvCtx.timeoutObj = new GridTimeoutObject() { + @Override public IgniteUuid timeoutId() { + return rcvCtx.sesId; + } + + @Override public long endTime() { + return U.currentTimeMillis() + ctx.config().getNetworkTimeout(); + } + + @Override public void onTimeout() { + ReceiverContext rcvCtx0 = rcvCtxs.remove(topic); + + closeRecevier(rcvCtx0, new IgniteCheckedException("Receiver is closed due to " + + "reconnect timeout has been occured")); + } + }); } catch (InterruptedException | ClassNotFoundException e) { throw new IgniteCheckedException(e); @@ -3006,6 +3027,9 @@ private static class ReceiverContext { /** Handler currently in use flag. */ private final AtomicBoolean active = new AtomicBoolean(); + /** Flag indicates that current file handling process must be interrupted. */ + private final AtomicBoolean interrupted = new AtomicBoolean(); + /** Flag indicates session started. */ private boolean sesStarted; @@ -3015,12 +3039,12 @@ private static class ReceiverContext { /** Last infinished downloading object. */ private AbstractReceiver rcv; - /** Flag indicates that current file handling process must be interrupted. */ - private volatile boolean interrupted; - /** Last saved state about file data processing. */ private TransmissionMeta lastState; + /** Close receiver timeout object. */ + private GridTimeoutObject timeoutObj; + /** * @param nodeId Remote node id. * @param hnd Channel handler of current topic. From 76fe688f368998f1917a1cc347879087a73bd0d7 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 19:53:56 +0300 Subject: [PATCH 35/69] IGNITE-10619: reformat internal exception throwing --- .../communication/AbstractReceiver.java | 11 ++++--- .../managers/communication/FileSender.java | 29 +++++++--------- .../managers/communication/GridIoManager.java | 33 +++++++++---------- 3 files changed, 33 insertions(+), 40 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java index 84c2879c7c234..81848a182c991 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java @@ -48,13 +48,14 @@ protected AbstractReceiver( * @throws IOException If an io exception occurred. * @throws IgniteCheckedException If some check failed. */ - public void receive(ReadableByteChannel ch) throws IOException, IgniteCheckedException { + public void receive(ReadableByteChannel ch) throws IOException, InterruptedException, IgniteCheckedException { // Read data from the input. while (hasNextChunk()) { - if (Thread.currentThread().isInterrupted() || stopped()) { - throw new IgniteCheckedException("Thread has been interrupted or operation has been cancelled " + - "due to node is stopping. Channel processing has been stopped."); - } + if (Thread.currentThread().isInterrupted()) + throw new InterruptedException("Recevier has been interrupted"); + + if (stopped()) + throw new IgniteCheckedException("Receiver has been cancelled. Channel processing has been stopped."); readChunk(ch); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 0fa9e21ce246e..93533e2396f6f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -68,7 +68,7 @@ class FileSender extends AbstractTransmission { * @param log Ignite logger. * @param factory Factory to produce IO interface on given file. * @param chunkSize Size of chunks. - * @throws IgniteCheckedException If fails. + * @throws IOException If fails. */ public FileSender( File file, @@ -80,7 +80,7 @@ public FileSender( IgniteLogger log, FileIOFactory factory, int chunkSize - ) throws IgniteCheckedException { + ) throws IOException { super(new TransmissionMeta(file.getName(), off, cnt, params, plc, null), stopChecker, log, chunkSize); assert file != null; @@ -88,15 +88,9 @@ public FileSender( this.file = file; this.factory = factory; - try { - // Can be not null if reconnection is going to be occurred. - if (fileIo == null) - fileIo = factory.create(file); - } - catch (IOException e) { - // Consider this IO exeption as a user one (not the network exception) and interrupt upload process. - throw new IgniteCheckedException("Unable to initialize source file. File sender upload will be stopped", e); - } + // Can be not null if reconnection is going to be occurred. + if (fileIo == null) + fileIo = factory.create(file); } /** @@ -104,12 +98,12 @@ public FileSender( * @param oo Channel to write meta info to. * @param rcvMeta Connection meta received. * @throws IOException If a transport exception occurred. - * @throws IgniteCheckedException If fails. + * @throws InterruptedException If thread interrupted. */ public void send(WritableByteChannel ch, ObjectOutput oo, @Nullable TransmissionMeta rcvMeta - ) throws IOException, IgniteCheckedException { + ) throws IOException, InterruptedException, IgniteCheckedException { // If not the initial connection for the current session. updateSenderState(rcvMeta); @@ -125,10 +119,11 @@ public void send(WritableByteChannel ch, null)); while (hasNextChunk()) { - if (Thread.currentThread().isInterrupted() || stopped()) { - throw new IgniteCheckedException("Thread has been interrupted or operation has been cancelled " + - "due to node is stopping. Channel processing has been stopped."); - } + if (Thread.currentThread().isInterrupted()) + throw new InterruptedException("Sender thread has been interruped"); + + if (stopped()) + throw new IgniteCheckedException("Sender has been cancelled due to the local node is stopping"); writeChunk(ch); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 739ba2cf3339c..e2e8bb0cf4109 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2805,7 +2805,7 @@ private void receiveFromChannel( ObjectInputStream in, ObjectOutputStream out, ReadableByteChannel ch - ) throws IgniteCheckedException { + ) throws IgniteCheckedException, InterruptedException, ClassNotFoundException { // Begin method must be called only once. if (!rcvCtx.sesStarted) { rcvCtx.hnd.onBegin(rcvCtx.nodeId); @@ -2900,9 +2900,6 @@ private void receiveFromChannel( } }); } - catch (InterruptedException | ClassNotFoundException e) { - throw new IgniteCheckedException(e); - } } /** @@ -3162,7 +3159,7 @@ public TransmissionSender( * @throws IgniteCheckedException If fails. * @throws IOException If fails. */ - private TransmissionMeta connect() throws IgniteCheckedException, IOException { + private TransmissionMeta connect() throws IgniteCheckedException, IOException, ClassNotFoundException { senderStopFlags.putIfAbsent(sesKey, new AtomicBoolean()); SocketChannel channel = (SocketChannel)openChannel(remoteId, @@ -3178,13 +3175,8 @@ private TransmissionMeta connect() throws IgniteCheckedException, IOException { TransmissionMeta syncMeta; - try { - // Synchronize state between remote and local nodes. - syncMeta = (TransmissionMeta)in.readObject(); - } - catch (ClassNotFoundException e) { - throw new IgniteCheckedException(e); - } + // Synchronize state between remote and local nodes. + syncMeta = (TransmissionMeta)in.readObject(); return syncMeta; } @@ -3199,7 +3191,7 @@ public void send( File file, Map params, TransmissionPolicy plc - ) throws IgniteCheckedException { + ) throws IgniteCheckedException, InterruptedException, IOException { send(file, 0, file.length(), params, plc); } @@ -3211,7 +3203,7 @@ public void send( public void send( File file, TransmissionPolicy plc - ) throws IgniteCheckedException { + ) throws IgniteCheckedException, InterruptedException, IOException { send(file, 0, file.length(), new HashMap<>(), plc); } @@ -3229,7 +3221,7 @@ public void send( long cnt, Map params, TransmissionPolicy plc - ) throws IgniteCheckedException { + ) throws IgniteCheckedException, InterruptedException, IOException { try (FileSender snd = new FileSender(file, offset, cnt, @@ -3304,11 +3296,16 @@ public void send( ", transferred=" + snd.transferred() + ", remoteId=" + remoteId +']'); } - catch (Exception e) { + catch (IgniteCheckedException | InterruptedException e) { + closeChannelQuiet(); + + throw e; + } + catch (Throwable t) { closeChannelQuiet(); - throw new IgniteCheckedException("Exception while uploading file to the remote node. The process stopped " + - "[remoteId=" + remoteId + ", file=" + file.getName() + ", sesKey=" + sesKey + ']', e); + throw new IgniteException("Exception while uploading file to the remote node. The process stopped " + + "[remoteId=" + remoteId + ", file=" + file.getName() + ", sesKey=" + sesKey + ']', t); } } From 4712bdf3bae7beab87b8a1b119bd83a92ba3fdf1 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 19:55:20 +0300 Subject: [PATCH 36/69] IGNITE-10619: add timeout object, close receiver only when it is need 2 --- .../GridIoManagerFileTransmissionSelfTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 2f5c526467a34..266b4e8c9914f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -529,7 +529,7 @@ public void testFileHandlerChannelCloseIfAnotherOpened() throws Exception { } }); - IgniteCheckedException[] errs = new IgniteCheckedException[1]; + Exception[] errs = new Exception[1]; try (GridIoManager.TransmissionSender sender = snd.context() .io() @@ -542,7 +542,7 @@ public void testFileHandlerChannelCloseIfAnotherOpened() throws Exception { try { sender.send(fileToSend, TransmissionPolicy.FILE); } - catch (IgniteCheckedException e) { + catch (IgniteCheckedException | IOException | InterruptedException e) { errs[0] = e; } finally { @@ -554,7 +554,7 @@ public void testFileHandlerChannelCloseIfAnotherOpened() throws Exception { try { anotherSender.send(fileToSend, TransmissionPolicy.FILE); } - catch (IgniteCheckedException e) { + catch (IgniteCheckedException | IOException | InterruptedException e) { errs[0] = e; } finally { From 237a4fbc7cfcbf06bbfccedb2f63ec6b1f7e2f76 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 20:48:43 +0300 Subject: [PATCH 37/69] IGNITE-10619: exception handling 2 --- .../internal/managers/communication/AbstractReceiver.java | 3 ++- .../internal/managers/communication/FileReceiver.java | 2 +- .../ignite/internal/managers/communication/FileSender.java | 6 ++++-- .../internal/managers/communication/GridIoManager.java | 4 ++-- .../GridIoManagerFileTransmissionSelfTest.java | 4 ++-- 5 files changed, 11 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java index 81848a182c991..b3057601c38bb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java @@ -22,6 +22,7 @@ import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.NodeStoppingException; /** * Class represents a receiver of data which can be pulled from a channel by chunks of @@ -55,7 +56,7 @@ public void receive(ReadableByteChannel ch) throws IOException, InterruptedExcep throw new InterruptedException("Recevier has been interrupted"); if (stopped()) - throw new IgniteCheckedException("Receiver has been cancelled. Channel processing has been stopped."); + throw new NodeStoppingException("Receiver has been cancelled. Channel processing has been stopped."); readChunk(ch); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index b4f44e4e65026..39d3546315fde 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -97,7 +97,7 @@ public FileReceiver( } /** {@inheritDoc} */ - @Override public void receive(ReadableByteChannel ch) throws IOException, IgniteCheckedException { + @Override public void receive(ReadableByteChannel ch) throws IOException, InterruptedException, IgniteCheckedException { super.receive(ch); if (transferred == meta.count()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 93533e2396f6f..94da0ff15f2d3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -27,6 +27,7 @@ import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -99,11 +100,12 @@ public FileSender( * @param rcvMeta Connection meta received. * @throws IOException If a transport exception occurred. * @throws InterruptedException If thread interrupted. + * @throws NodeStoppingException If stopping. */ public void send(WritableByteChannel ch, ObjectOutput oo, @Nullable TransmissionMeta rcvMeta - ) throws IOException, InterruptedException, IgniteCheckedException { + ) throws IOException, InterruptedException, NodeStoppingException { // If not the initial connection for the current session. updateSenderState(rcvMeta); @@ -123,7 +125,7 @@ public void send(WritableByteChannel ch, throw new InterruptedException("Sender thread has been interruped"); if (stopped()) - throw new IgniteCheckedException("Sender has been cancelled due to the local node is stopping"); + throw new NodeStoppingException("Sender has been cancelled due to the local node is stopping"); writeChunk(ch); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index e2e8bb0cf4109..56531b50a616d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -3283,7 +3283,7 @@ public void send( retries++; if (retries == retryCnt) { - throw new IgniteCheckedException("The number of retry attempts to upload file exceeded " + + throw new IgniteException("The number of retry attempts to upload file exceeded " + "the limit: " + retryCnt, e); } } @@ -3304,7 +3304,7 @@ public void send( catch (Throwable t) { closeChannelQuiet(); - throw new IgniteException("Exception while uploading file to the remote node. The process stopped " + + throw new IgniteException("Exception while sending file to the remote node. The process stopped " + "[remoteId=" + remoteId + ", file=" + file.getName() + ", sesKey=" + sesKey + ']', t); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 266b4e8c9914f..77291596f43b2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -459,7 +459,7 @@ public void testFileHandlerNextWriterOpened() throws Exception { } }); - IgniteCheckedException expectedErr = null; + Exception expectedErr = null; try (GridIoManager.TransmissionSender sender = snd.context() .io() @@ -492,7 +492,7 @@ public void testFileHandlerNextWriterOpened() throws Exception { .openTransmissionSender(rcv.localNode().id(), topic)) { sender.send(fileToSend, TransmissionPolicy.FILE); } - catch (IgniteCheckedException e) { + catch (IgniteException e) { // Must catch execption here. expectedErr = e; } From 69df21a43031fd8a3e8dc115a9e26291938a3f8b Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 20:54:59 +0300 Subject: [PATCH 38/69] IGNITE-10619: exception handling 3 --- .../managers/communication/GridIoManager.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 56531b50a616d..7bd5f80017f5b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -933,7 +933,7 @@ private void format(StringBuilder b, Collection> pairs, SimpleD if (nodeId.equals(e.getValue().nodeId)) { it.remove(); - closeRecevier(e.getValue(), + interruptRecevier(e.getValue(), new ClusterTopologyCheckedException("Remove node left the grid. " + "Receiver has been stopped : " + nodeId)); } @@ -1107,7 +1107,7 @@ private void format(StringBuilder b, Collection> pairs, SimpleD topicTransmissionHnds.clear(); for (ReceiverContext rctx : rcvCtxs.values()) { - closeRecevier(rctx, new NodeStoppingException("Local node io manager requested to be stopped: " + interruptRecevier(rctx, new NodeStoppingException("Local node io manager requested to be stopped: " + ctx.localNodeId())); } @@ -1878,7 +1878,7 @@ public void addTransmissionHandler(Object topic, TransmissionHandler hnd) { public void removeTransmissionHandler(Object topic) { topicTransmissionHnds.remove(topic); - closeRecevier(rcvCtxs.remove(topic), + interruptRecevier(rcvCtxs.remove(topic), new IgniteCheckedException("Receiver has been closed due to removing corresponding transmission handler " + "on local node [nodeId=" + ctx.localNodeId() + ']')); } @@ -2671,7 +2671,7 @@ public int getOutboundMessagesQueueSize() { * @param ctx Receiver context to use. * @param ex Exception to close receiver with. */ - private static void closeRecevier(ReceiverContext ctx, IgniteCheckedException ex) { + private static void interruptRecevier(ReceiverContext ctx, IgniteCheckedException ex) { if (ctx == null) return; @@ -2764,7 +2764,7 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { if (!newSesId.equals(rcvCtx.sesId)) { // Attempt to receive file with new session id. Context must be reinited, // previous session must be failed. - closeRecevier(rcvCtx, new IgniteCheckedException("Process has been aborted " + + interruptRecevier(rcvCtx, new IgniteCheckedException("Process has been aborted " + "by transfer attempt with a new session [sesId=" + newSesId + ", nodeId=" + nodeId + ']')); rcvCtx = new ReceiverContext(nodeId, hnd, newSesId); @@ -2782,7 +2782,7 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { catch (Throwable t) { U.error(log, "Download session cannot be finished due to an unexpected error [ctx=" + rcvCtx + ']', t); - closeRecevier(rcvCtx, new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t)); + interruptRecevier(rcvCtx, new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t)); } finally { if (rcvCtx != null) @@ -2895,7 +2895,7 @@ private void receiveFromChannel( @Override public void onTimeout() { ReceiverContext rcvCtx0 = rcvCtxs.remove(topic); - closeRecevier(rcvCtx0, new IgniteCheckedException("Receiver is closed due to " + + interruptRecevier(rcvCtx0, new IgniteCheckedException("Receiver is closed due to " + "reconnect timeout has been occured")); } }); From ce377d934d478761f7e32e94a6faaa5443fe5d54 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 21:10:17 +0300 Subject: [PATCH 39/69] IGNITE-10619: exception handling 4 --- .../communication/AbstractReceiver.java | 7 ++-- .../managers/communication/ChunkReceiver.java | 9 ++-- .../managers/communication/FileReceiver.java | 18 ++++---- .../managers/communication/GridIoManager.java | 6 +-- .../communication/TransmissionHandler.java | 11 ++--- ...GridIoManagerFileTransmissionSelfTest.java | 41 ++++++++----------- 6 files changed, 37 insertions(+), 55 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java index b3057601c38bb..dcd19af12e8e1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java @@ -47,9 +47,9 @@ protected AbstractReceiver( /** * @param ch Input channel to read data from. * @throws IOException If an io exception occurred. - * @throws IgniteCheckedException If some check failed. + * @throws NodeStoppingException If some check failed. */ - public void receive(ReadableByteChannel ch) throws IOException, InterruptedException, IgniteCheckedException { + public void receive(ReadableByteChannel ch) throws IOException, InterruptedException, NodeStoppingException { // Read data from the input. while (hasNextChunk()) { if (Thread.currentThread().isInterrupted()) @@ -68,7 +68,6 @@ public void receive(ReadableByteChannel ch) throws IOException, InterruptedExcep /** * @param ch Channel to read data from. * @throws IOException If fails. - * @throws IgniteCheckedException If fails. */ - protected abstract void readChunk(ReadableByteChannel ch) throws IOException, IgniteCheckedException; + protected abstract void readChunk(ReadableByteChannel ch) throws IOException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java index a134a7570f26f..5b1da19016093 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java @@ -22,9 +22,8 @@ import java.nio.ByteOrder; import java.nio.channels.ReadableByteChannel; import java.util.function.BooleanSupplier; -import org.apache.ignite.IgniteCheckedException; +import java.util.function.Consumer; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; @@ -34,7 +33,7 @@ */ class ChunkReceiver extends AbstractReceiver { /** Chunked channel handler to process data with chunks. */ - private final IgniteThrowableConsumer hnd; + private final Consumer hnd; /** The destination object to transfer data to\from. */ private ByteBuffer buf; @@ -50,7 +49,7 @@ public ChunkReceiver( TransmissionMeta meta, int chunkSize, BooleanSupplier stopChecker, - IgniteThrowableConsumer hnd, + Consumer hnd, IgniteLogger log ) { super(meta, stopChecker, log, chunkSize); @@ -64,7 +63,7 @@ public ChunkReceiver( } /** {@inheritDoc} */ - @Override protected void readChunk(ReadableByteChannel ch) throws IOException, IgniteCheckedException { + @Override protected void readChunk(ReadableByteChannel ch) throws IOException { assert buf != null : "Buffer is used to deilver readed data to the used and cannot be null: " + this; buf.rewind(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index 39d3546315fde..e6e8db13b1367 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -23,13 +23,13 @@ import java.nio.channels.ReadableByteChannel; import java.nio.file.Files; import java.util.function.BooleanSupplier; +import java.util.function.Consumer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.apache.ignite.internal.util.lang.IgniteOutClosureX; -import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; @@ -48,7 +48,7 @@ class FileReceiver extends AbstractReceiver { private final FileIOFactory factory; /** Handler to notify when a file has been processed. */ - private final IgniteOutClosureX> hndProvider; + private final Consumer hnd; /** The abstract java representation of the chunked file. */ private File file; @@ -61,7 +61,7 @@ class FileReceiver extends AbstractReceiver { * @param meta Initial file meta info. * @param stopChecker Node stop or prcoess interrupt checker. * @param factory Factory to produce IO interface on files. - * @param hndProvider Transmission handler provider to process download result. + * @param hnd Transmission handler provider to process download result. * @param path File path to destination receiver source. * @param log Ignite logger. * @throws IgniteCheckedException If fails. @@ -71,18 +71,18 @@ public FileReceiver( int chunkSize, BooleanSupplier stopChecker, FileIOFactory factory, - IgniteOutClosureX> hndProvider, + Consumer hnd, String path, IgniteLogger log ) throws IgniteCheckedException { super(meta, stopChecker, log, chunkSize); - A.notNull(hndProvider, "FileHandler must be provided by transmission handler"); + A.notNull(hnd, "FileHandler must be provided by transmission handler"); A.notNull(path, "File absolute path cannot be null"); A.ensure(!path.trim().isEmpty(), "File absolute path cannot be empty "); this.factory = factory; - this.hndProvider = hndProvider; + this.hnd = hnd; file = new File(path); @@ -97,11 +97,11 @@ public FileReceiver( } /** {@inheritDoc} */ - @Override public void receive(ReadableByteChannel ch) throws IOException, InterruptedException, IgniteCheckedException { + @Override public void receive(ReadableByteChannel ch) throws IOException, InterruptedException, NodeStoppingException { super.receive(ch); if (transferred == meta.count()) - hndProvider.apply().accept(file); + hnd.accept(file); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 7bd5f80017f5b..02001db38b48e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2952,11 +2952,7 @@ private AbstractReceiver createReceiver( chunkSize, stopChecker, fileIoFactory, - new IgniteOutClosureX>() { - @Override public IgniteThrowableConsumer applyx() throws IgniteCheckedException { - return hnd.fileHandler(nodeId, meta); - } - }, + hnd.fileHandler(nodeId, meta), hnd.filePath(nodeId, meta), log); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java index 67cf7df2372e4..1e0d1748606a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java @@ -21,8 +21,7 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.UUID; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; +import java.util.function.Consumer; /** * Class represents a handler for the set of files considered to be transferred from the remote node. This handler @@ -70,10 +69,8 @@ public interface TransmissionHandler { * @param nodeId Remote node id from which request has been received. * @param initMeta Initial handler meta info. * @return Instance of chunk handler to process incoming data by chunks. - * @throws IgniteCheckedException If fails. */ - public IgniteThrowableConsumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) - throws IgniteCheckedException; + public Consumer chunkHandler(UUID nodeId, TransmissionMeta initMeta); /** * File handler represents by itself the way of input data stream processing. All the data will @@ -83,8 +80,6 @@ public IgniteThrowableConsumer chunkHandler(UUID nodeId, Transmissio * @param nodeId Remote node id from which request has been received. * @param initMeta Initial handler meta info. * @return Intance of read handler to process incoming data like the {@link FileChannel} manner. - * @throws IgniteCheckedException If fails. */ - public IgniteThrowableConsumer fileHandler(UUID nodeId, TransmissionMeta initMeta) - throws IgniteCheckedException; + public Consumer fileHandler(UUID nodeId, TransmissionMeta initMeta); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 77291596f43b2..08c3798831359 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -36,6 +36,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.LongAdder; +import java.util.function.Consumer; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; @@ -53,7 +54,6 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; -import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -91,10 +91,10 @@ public class GridIoManagerFileTransmissionSelfTest extends GridCommonAbstractTes private File tempStore; /** - * @throws Exception If fails. + * Called before tests started. */ @BeforeClass - public static void beforeAll() throws Exception { + public static void beforeAll() { topic = GridTopic.TOPIC_CACHE.topic("test", 0); fileBinFilter = new FilenameFilter() { @@ -115,10 +115,10 @@ public void before() throws Exception { } /** - * @throws Exception if failed. + * Called after test run. */ @After - public void after() throws Exception { + public void after() { stopAllGrids(); U.closeQuiet(fileIo[0]); @@ -165,9 +165,8 @@ public void testFileHandlerBase() throws Exception { return new File(tempStore, fileMeta.name()).getAbsolutePath(); } - @Override public IgniteThrowableConsumer fileHandler(UUID nodeId, TransmissionMeta initMeta) { - return new IgniteThrowableConsumer() { - + @Override public Consumer fileHandler(UUID nodeId, TransmissionMeta initMeta) { + return new Consumer() { @Override public void accept(File file) { assertTrue(fileSizes.containsKey(file.getName())); // Save all params. @@ -304,8 +303,6 @@ public void testFileHandlerOnReceiverLeft() throws Exception { */ @Test public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { - final CountDownLatch latch = new CountDownLatch(1); - IgniteEx snd = startGrid(0); IgniteEx rcv = startGrid(1); @@ -618,8 +615,7 @@ public void testChunkHandlerWithReconnect() throws Exception { } /** {@inheritDoc} */ - @Override public IgniteThrowableConsumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) - throws IgniteCheckedException { + @Override public Consumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) { if (fileIo[0] == null) { try { @@ -627,14 +623,14 @@ public void testChunkHandlerWithReconnect() throws Exception { fileIo[0].position(initMeta.offset()); } catch (IOException e) { - throw new IgniteCheckedException(e); + throw new IgniteException(e); } } - return new IgniteThrowableConsumer() { + return new Consumer() { final LongAdder transferred = new LongAdder(); - @Override public void accept(ByteBuffer buff) throws IgniteCheckedException { + @Override public void accept(ByteBuffer buff) { try { assertTrue(buff.order() == ByteOrder.nativeOrder()); assertEquals(0, buff.position()); @@ -646,7 +642,7 @@ public void testChunkHandlerWithReconnect() throws Exception { transferred.add(buff.capacity()); } catch (Throwable e) { - throw new IgniteCheckedException(e); + throw new IgniteException(e); } finally { closeIfTransferred(); @@ -692,7 +688,7 @@ public void testChunkHandlerInitSizeFail() throws Exception { rcv.context().io().addTransmissionHandler(topic, new TransmissionHandlerAdapter() { /** {@inheritDoc} */ - @Override public IgniteThrowableConsumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) { + @Override public Consumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) { throw new IgniteException("Test exception. Initialization failed"); } }); @@ -799,9 +795,8 @@ public DefaultTransmissionHandler(IgniteEx rcv, File fileToSend, File tempStorag } /** {@inheritDoc} */ - @Override public IgniteThrowableConsumer fileHandler(UUID nodeId, TransmissionMeta initMeta) - throws IgniteCheckedException { - return new IgniteThrowableConsumer() { + @Override public Consumer fileHandler(UUID nodeId, TransmissionMeta initMeta) { + return new Consumer() { @Override public void accept(File file) { assertEquals(fileToSend.length(), file.length()); assertCrcEquals(fileToSend, file); @@ -825,14 +820,12 @@ private static class TransmissionHandlerAdapter implements TransmissionHandler { } /** {@inheritDoc} */ - @Override public IgniteThrowableConsumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) - throws IgniteCheckedException { + @Override public Consumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) { return null; } /** {@inheritDoc} */ - @Override public IgniteThrowableConsumer fileHandler(UUID nodeId, TransmissionMeta initMeta) - throws IgniteCheckedException { + @Override public Consumer fileHandler(UUID nodeId, TransmissionMeta initMeta) { return null; } From e18dcfa09fd74dc4da8cda6343e3408188eeb8df Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 21:45:10 +0300 Subject: [PATCH 40/69] IGNITE-10619: exception handling 5 --- .../communication/AbstractReceiver.java | 8 +++--- .../managers/communication/FileReceiver.java | 11 +------- .../managers/communication/FileSender.java | 8 +++--- .../managers/communication/GridIoManager.java | 25 +++++++++++-------- ...GridIoManagerFileTransmissionSelfTest.java | 16 ++++++------ 5 files changed, 30 insertions(+), 38 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java index dcd19af12e8e1..b1e8529cd76f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java @@ -20,9 +20,8 @@ import java.io.IOException; import java.nio.channels.ReadableByteChannel; import java.util.function.BooleanSupplier; -import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.NodeStoppingException; /** * Class represents a receiver of data which can be pulled from a channel by chunks of @@ -47,16 +46,15 @@ protected AbstractReceiver( /** * @param ch Input channel to read data from. * @throws IOException If an io exception occurred. - * @throws NodeStoppingException If some check failed. */ - public void receive(ReadableByteChannel ch) throws IOException, InterruptedException, NodeStoppingException { + public void receive(ReadableByteChannel ch) throws IOException, InterruptedException { // Read data from the input. while (hasNextChunk()) { if (Thread.currentThread().isInterrupted()) throw new InterruptedException("Recevier has been interrupted"); if (stopped()) - throw new NodeStoppingException("Receiver has been cancelled. Channel processing has been stopped."); + throw new IgniteException("Receiver has been cancelled. Channel processing has been stopped."); readChunk(ch); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index e6e8db13b1367..ba93ee828aa07 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -29,7 +29,6 @@ import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; -import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; @@ -40,13 +39,6 @@ * {@link FileChannel#transferFrom(ReadableByteChannel, long, long)}. */ class FileReceiver extends AbstractReceiver { - /** Future will be done when receiver is closed. */ - protected final GridFutureAdapter closeFut = new GridFutureAdapter<>(); - - /** The default factory to provide IO oprations over underlying file. */ - @GridToStringExclude - private final FileIOFactory factory; - /** Handler to notify when a file has been processed. */ private final Consumer hnd; @@ -81,7 +73,6 @@ public FileReceiver( A.notNull(path, "File absolute path cannot be null"); A.ensure(!path.trim().isEmpty(), "File absolute path cannot be empty "); - this.factory = factory; this.hnd = hnd; file = new File(path); @@ -97,7 +88,7 @@ public FileReceiver( } /** {@inheritDoc} */ - @Override public void receive(ReadableByteChannel ch) throws IOException, InterruptedException, NodeStoppingException { + @Override public void receive(ReadableByteChannel ch) throws IOException, InterruptedException { super.receive(ch); if (transferred == meta.count()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 94da0ff15f2d3..c9b9ea79f3bad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -25,9 +25,8 @@ import java.nio.channels.WritableByteChannel; import java.util.Map; import java.util.function.BooleanSupplier; -import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -100,12 +99,11 @@ public FileSender( * @param rcvMeta Connection meta received. * @throws IOException If a transport exception occurred. * @throws InterruptedException If thread interrupted. - * @throws NodeStoppingException If stopping. */ public void send(WritableByteChannel ch, ObjectOutput oo, @Nullable TransmissionMeta rcvMeta - ) throws IOException, InterruptedException, NodeStoppingException { + ) throws IOException, InterruptedException { // If not the initial connection for the current session. updateSenderState(rcvMeta); @@ -125,7 +123,7 @@ public void send(WritableByteChannel ch, throw new InterruptedException("Sender thread has been interruped"); if (stopped()) - throw new NodeStoppingException("Sender has been cancelled due to the local node is stopping"); + throw new IgniteException("Sender has been cancelled due to the local node is stopping"); writeChunk(ch); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 02001db38b48e..ba3508e1ad2b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -286,10 +286,7 @@ public class GridIoManager extends GridManagerAdapter lsnrMap = new ConcurrentHashMap<>(); @@ -2805,7 +2802,7 @@ private void receiveFromChannel( ObjectInputStream in, ObjectOutputStream out, ReadableByteChannel ch - ) throws IgniteCheckedException, InterruptedException, ClassNotFoundException { + ) throws IgniteCheckedException, InterruptedException { // Begin method must be called only once. if (!rcvCtx.sesStarted) { rcvCtx.hnd.onBegin(rcvCtx.nodeId); @@ -2871,6 +2868,9 @@ private void receiveFromChannel( } } } + catch (ClassNotFoundException e) { + throw new IgniteException(e); + } catch (IOException e) { // Waiting for re-establishing connection. U.warn(log, "Сonnection from the remote node lost. Will wait for the new one to continue file " + @@ -2949,7 +2949,7 @@ private AbstractReceiver createReceiver( case FILE: return new FileReceiver( meta, - chunkSize, + DFLT_CHUNK_SIZE_BYTES, stopChecker, fileIoFactory, hnd.fileHandler(nodeId, meta), @@ -3155,7 +3155,7 @@ public TransmissionSender( * @throws IgniteCheckedException If fails. * @throws IOException If fails. */ - private TransmissionMeta connect() throws IgniteCheckedException, IOException, ClassNotFoundException { + private TransmissionMeta connect() throws IgniteCheckedException, IOException { senderStopFlags.putIfAbsent(sesKey, new AtomicBoolean()); SocketChannel channel = (SocketChannel)openChannel(remoteId, @@ -3171,8 +3171,13 @@ private TransmissionMeta connect() throws IgniteCheckedException, IOException, C TransmissionMeta syncMeta; - // Synchronize state between remote and local nodes. - syncMeta = (TransmissionMeta)in.readObject(); + try { + // Synchronize state between remote and local nodes. + syncMeta = (TransmissionMeta)in.readObject(); + } + catch (ClassNotFoundException e) { + throw new IgniteException (e); + } return syncMeta; } @@ -3226,7 +3231,7 @@ public void send( () -> stopping || senderStopFlags.get(sesKey).get(), log, fileIoFactory, - chunkSize) + DFLT_CHUNK_SIZE_BYTES) ) { if (log.isDebugEnabled()) { log.debug("Start writing file to remote node [file=" + file.getName() + diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 08c3798831359..a4969ff7b2cba 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -194,10 +194,7 @@ public void testFileHandlerBase() throws Exception { params.put(file.getName(), file.hashCode()); - sender.send(file, - // Put additional params to map. - params, - TransmissionPolicy.FILE); + sender.send(file, params, TransmissionPolicy.FILE); } } @@ -258,7 +255,7 @@ public void testFileHandlerOnBeginFails() throws Exception { /** * @throws Exception If fails. */ - @Test(expected = IgniteCheckedException.class) + @Test(expected = IgniteException.class) public void testFileHandlerOnReceiverLeft() throws Exception { final int fileSizeBytes = 5 * 1024 * 1024; final AtomicInteger chunksCnt = new AtomicInteger(); @@ -331,16 +328,19 @@ public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore)); + Exception err = null; + try (GridIoManager.TransmissionSender sender = snd.context() .io() .openTransmissionSender(rcv.localNode().id(), topic)) { sender.send(fileToSend, TransmissionPolicy.FILE); } - catch (IgniteCheckedException e) { + catch (Exception e) { // Ignore node stopping exception. - U.log(log,"Expected node stopping exception", e); + err = e; } + assertEquals(IgniteException.class, err.getClass()); assertEquals("Uncomplete resources must be cleaned up on sender left", 1, // only fileToSend is expected to exist fileCount(tempStore.toPath())); @@ -396,7 +396,7 @@ public void testFileHandlerReconnectOnReadFail() throws Exception { * @throws Exception If fails. */ @Test(expected = IgniteCheckedException.class) - public void testFileHandlerReconnectOnInitFail() throws Exception { + public void testFileHandlerSenderStoppedIfReceiverInitFail() throws Exception { final int fileSizeBytes = 5 * 1024 * 1024; final AtomicBoolean throwFirstTime = new AtomicBoolean(); From b5910c0b0ac0b5a6914709df1e7f59fa3c4f70ba Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 21:52:28 +0300 Subject: [PATCH 41/69] IGNITE-10619: rename to transmission receiver --- .../internal/managers/communication/ChunkReceiver.java | 2 +- .../internal/managers/communication/FileReceiver.java | 9 ++++----- .../internal/managers/communication/GridIoManager.java | 6 ++---- .../{AbstractReceiver.java => TransmissionReceiver.java} | 4 ++-- 4 files changed, 9 insertions(+), 12 deletions(-) rename modules/core/src/main/java/org/apache/ignite/internal/managers/communication/{AbstractReceiver.java => TransmissionReceiver.java} (96%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java index 5b1da19016093..2d19966c928fc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java @@ -31,7 +31,7 @@ * Buffered chunked receiver can handle input socket channel by chunks of data and * deliver it to an allocated {@link ByteBuffer}. */ -class ChunkReceiver extends AbstractReceiver { +class ChunkReceiver extends TransmissionReceiver { /** Chunked channel handler to process data with chunks. */ private final Consumer hnd; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index ba93ee828aa07..26466155fe2fa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -25,8 +25,8 @@ import java.util.function.BooleanSupplier; import java.util.function.Consumer; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -38,7 +38,7 @@ * Class represents a chunk data receiver which is pulling data from channel vi * {@link FileChannel#transferFrom(ReadableByteChannel, long, long)}. */ -class FileReceiver extends AbstractReceiver { +class FileReceiver extends TransmissionReceiver { /** Handler to notify when a file has been processed. */ private final Consumer hnd; @@ -56,7 +56,6 @@ class FileReceiver extends AbstractReceiver { * @param hnd Transmission handler provider to process download result. * @param path File path to destination receiver source. * @param log Ignite logger. - * @throws IgniteCheckedException If fails. */ public FileReceiver( TransmissionMeta meta, @@ -66,7 +65,7 @@ public FileReceiver( Consumer hnd, String path, IgniteLogger log - ) throws IgniteCheckedException { + ) { super(meta, stopChecker, log, chunkSize); A.notNull(hnd, "FileHandler must be provided by transmission handler"); @@ -83,7 +82,7 @@ public FileReceiver( fileIo.position(meta.offset()); } catch (IOException e) { - throw new IgniteCheckedException("Unable to open destination file. Receiver will will be stopped", e); + throw new IgniteException("Unable to open destination file. Receiver will will be stopped", e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index ba3508e1ad2b7..697bdee803d18 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -105,9 +105,7 @@ import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridTuple3; -import org.apache.ignite.internal.util.lang.IgniteOutClosureX; import org.apache.ignite.internal.util.lang.IgnitePair; -import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; @@ -2939,7 +2937,7 @@ void transfererFileIoFactory(FileIOFactory factory) { * @return Chunk data recevier. * @throws IgniteCheckedException If fails. */ - private AbstractReceiver createReceiver( + private TransmissionReceiver createReceiver( UUID nodeId, TransmissionHandler hnd, TransmissionMeta meta, @@ -3030,7 +3028,7 @@ private static class ReceiverContext { private int retries; /** Last infinished downloading object. */ - private AbstractReceiver rcv; + private TransmissionReceiver rcv; /** Last saved state about file data processing. */ private TransmissionMeta lastState; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionReceiver.java similarity index 96% rename from modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java rename to modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionReceiver.java index b1e8529cd76f2..fc72b2f735e4a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionReceiver.java @@ -27,14 +27,14 @@ * Class represents a receiver of data which can be pulled from a channel by chunks of * predefined size. Closes when a transmission of represented object ends. */ -abstract class AbstractReceiver extends AbstractTransmission { +abstract class TransmissionReceiver extends AbstractTransmission { /** * @param meta Initial file meta info. * @param stopChecker Node stop or prcoess interrupt checker. * @param log Ignite logger. * @param chunkSize Size of chunks. */ - protected AbstractReceiver( + protected TransmissionReceiver( TransmissionMeta meta, BooleanSupplier stopChecker, IgniteLogger log, From 1ac998dc6eed5c35c99203d14f45c6e02fb38529 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 21:57:41 +0300 Subject: [PATCH 42/69] IGNITE-10619: download to separate directory --- .../GridIoManagerFileTransmissionSelfTest.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index a4969ff7b2cba..01895d8a93d73 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -306,6 +306,7 @@ public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { snd.cluster().active(true); File fileToSend = createFileRandomData("tempFile15Mb", 15 * 1024 * 1024); + File downloadTo = U.resolveWorkDirectory(tempStore.getAbsolutePath(), "download", true); snd.context().io().transfererFileIoFactory(new FileIOFactory() { @Override public FileIO create(File file, OpenOption... modes) throws IOException { @@ -326,7 +327,12 @@ public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { } }); - rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore)); + rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore){ + /** {@inheritDoc} */ + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { + return new File(downloadTo, fileMeta.name()).getAbsolutePath(); + } + }); Exception err = null; @@ -342,8 +348,8 @@ public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { assertEquals(IgniteException.class, err.getClass()); assertEquals("Uncomplete resources must be cleaned up on sender left", - 1, // only fileToSend is expected to exist - fileCount(tempStore.toPath())); + 0, + fileCount(downloadTo.toPath())); } /** From f8629b16bff64021e09b4559a06660ee3173b62e Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 5 Aug 2019 22:04:34 +0300 Subject: [PATCH 43/69] IGNITE-10619: fail fast if incorrect method called --- .../managers/communication/FileSender.java | 2 +- ...GridIoManagerFileTransmissionSelfTest.java | 24 +++++++++++++++++++ 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index c9b9ea79f3bad..511557998179e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -173,7 +173,7 @@ private void writeChunk(WritableByteChannel ch) throws IOException { } /** {@inheritDoc} */ - @Override public void close() throws IOException { + @Override public void close() { U.closeQuiet(fileIo); fileIo = null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 01895d8a93d73..c53328bf2a931 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -240,6 +240,30 @@ public void testFileHandlerOnBeginFails() throws Exception { throw new IgniteException(exTestMessage); } + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { + fail("filePath must never be called"); + + return null; + } + + @Override public Consumer fileHandler(UUID nodeId, TransmissionMeta initMeta) { + fail("fileHandler must never be called"); + + return super.fileHandler(nodeId, initMeta); + } + + @Override public Consumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) { + fail("chunkHandler must never be called"); + + return super.chunkHandler(nodeId, initMeta); + } + + @Override public void onEnd(UUID nodeId) { + fail("onEnd must never be called"); + + super.onEnd(nodeId); + } + @Override public void onException(UUID nodeId, Throwable err) { assertEquals(exTestMessage, err.getMessage()); } From 73e2c74d0d772ebff2d0bebdefbb32eb60ecc3bc Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 11:40:04 +0300 Subject: [PATCH 44/69] IGNITE-10619: reformat throwing exceptions --- .../managers/communication/FileSender.java | 10 ---- .../managers/communication/GridIoManager.java | 48 +++++++++---------- 2 files changed, 22 insertions(+), 36 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 511557998179e..1bc884d8ec04d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -47,13 +47,6 @@ * @see FileChannel#transferTo(long, long, WritableByteChannel) */ class FileSender extends AbstractTransmission { - /** Default factory to provide IO oprations over given file. */ - @GridToStringExclude - private final FileIOFactory factory; - - /** File which will be send to remote by chunks. */ - private final File file; - /** Corresponding file channel to work with given file. */ @GridToStringExclude private FileIO fileIo; @@ -85,9 +78,6 @@ public FileSender( assert file != null; - this.file = file; - this.factory = factory; - // Can be not null if reconnection is going to be occurred. if (fileIo == null) fileIo = factory.create(file); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 697bdee803d18..3e1b031cc5b10 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -111,6 +111,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -2792,7 +2793,8 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { /** * @param topic Topic handler related to. * @param rcvCtx Receiver read context. - * @throws IgniteCheckedException If processing fails. + * @throws NodeStoppingException If processing fails. + * @throws InterruptedException If thread interrupted. */ private void receiveFromChannel( Object topic, @@ -2800,7 +2802,7 @@ private void receiveFromChannel( ObjectInputStream in, ObjectOutputStream out, ReadableByteChannel ch - ) throws IgniteCheckedException, InterruptedException { + ) throws NodeStoppingException, InterruptedException { // Begin method must be called only once. if (!rcvCtx.sesStarted) { rcvCtx.hnd.onBegin(rcvCtx.nodeId); @@ -2850,11 +2852,9 @@ private void receiveFromChannel( rcvCtx.rcv.close(); - long downloadTime = U.currentTimeMillis() - startTime; - U.log(log, "File has been received " + "[name=" + rcvCtx.rcv.state().name() + ", transferred=" + rcvCtx.rcv.transferred() + - ", time=" + (double)((downloadTime) / 1000) + " sec" + + ", time=" + (double)((U.currentTimeMillis() - startTime) / 1000) + " sec" + ", retries=" + rcvCtx.retries + ", remoteId=" + rcvCtx.nodeId + ']'); rcvCtx.rcv = null; @@ -2877,7 +2877,7 @@ private void receiveFromChannel( rcvCtx.retries++; if (rcvCtx.retries == retryCnt) { - throw new IgniteCheckedException("Number of retry attempts to download file exceeded the limit. " + + throw new IgniteException("Number of retry attempts to download file exceeded the limit. " + "Max attempts: " + retryCnt, e); } @@ -2903,19 +2903,18 @@ private void receiveFromChannel( /** * @param prev Previous available transmission meta. * @param next Next transmission meta. - * @throws IgniteCheckedException If fails. */ - private void validate(TransmissionMeta prev, TransmissionMeta next) throws IgniteCheckedException { - assertParameter(prev.name().equals(next.name()), "Attempt to load different file " + + private void validate(TransmissionMeta prev, TransmissionMeta next) { + A.ensure(prev.name().equals(next.name()), "Attempt to load different file " + "[prev=" + prev + ", next=" + next + ']'); - assertParameter(prev.offset() == next.offset(), + A.ensure(prev.offset() == next.offset(), "The next chunk offest is incorrect [prev=" + prev + ", meta=" + next + ']'); - assertParameter(prev.count() == next.count(), " The count of bytes to transfer for " + + A.ensure(prev.count() == next.count(), " The count of bytes to transfer for " + "the next chunk is incorrect [prev=" + prev + ", next=" + next + ']'); - assertParameter(prev.policy() == next.policy(), "Attemt to continue file upload with" + + A.ensure(prev.policy() == next.policy(), "Attemt to continue file upload with" + " different transmission policy [prev=" + prev + ", next=" + next + ']'); } @@ -2935,14 +2934,13 @@ void transfererFileIoFactory(FileIOFactory factory) { * @param meta Meta information about file pending to receive to create appropriate receiver. * @param stopChecker Process interrupt checker. * @return Chunk data recevier. - * @throws IgniteCheckedException If fails. */ private TransmissionReceiver createReceiver( UUID nodeId, TransmissionHandler hnd, TransmissionMeta meta, BooleanSupplier stopChecker - ) throws IgniteCheckedException { + ) { switch (meta.policy()) { case FILE: return new FileReceiver( @@ -2965,7 +2963,7 @@ private TransmissionReceiver createReceiver( log); default: - throw new IgniteCheckedException("The type of read plc is unknown. The impelentation " + + throw new IgniteException("The type of transmission policy is unknown. An impelentation " + "required: " + meta.policy()); } } @@ -3272,26 +3270,24 @@ public void send( catch (IOException e) { closeChannelQuiet(); - // Re-establish the new connection to continue upload. - U.warn(log, "Connection lost while writing file to remote node and " + - "will be re-establishing [remoteId=" + remoteId + ", file=" + file.getName() + - ", sesKey=" + sesKey + ", retries=" + retries + - ", transferred=" + snd.transferred() + - ", total=" + snd.state().count() + ']', e); - retries++; if (retries == retryCnt) { throw new IgniteException("The number of retry attempts to upload file exceeded " + "the limit: " + retryCnt, e); } + + // Re-establish the new connection to continue upload. + U.warn(log, "Connection lost while writing a file to remote node and " + + "will be reestablished [remoteId=" + remoteId + ", file=" + file.getName() + + ", sesKey=" + sesKey + ", retries=" + retries + + ", transferred=" + snd.transferred() + + ", total=" + snd.state().count() + ']', e); } } - long uploadTime = U.currentTimeMillis() - startTime; - - U.log(log, "File has been sent [name=" + file.getName() + - ", uploadTime=" + (double)((uploadTime) / 1000) + " sec, retries=" + retries + + U.log(log, "File has been sent to remote node [name=" + file.getName() + + ", uploadTime=" + (double)((U.currentTimeMillis() - startTime) / 1000) + " sec, retries=" + retries + ", transferred=" + snd.transferred() + ", remoteId=" + remoteId +']'); } From 2a017bc8a7acab9209887c91333d12e59ff008c7 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 17:31:04 +0300 Subject: [PATCH 45/69] IGNITE-10619: add lock to receiver context --- .../managers/communication/FileReceiver.java | 4 +- .../managers/communication/GridIoManager.java | 190 ++++++++++-------- .../timeout/GridTimeoutProcessor.java | 4 +- ...GridIoManagerFileTransmissionSelfTest.java | 105 +++++++++- 4 files changed, 214 insertions(+), 89 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index 26466155fe2fa..fc3b8b37931de 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -24,7 +24,6 @@ import java.nio.file.Files; import java.util.function.BooleanSupplier; import java.util.function.Consumer; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; @@ -102,6 +101,9 @@ public FileReceiver( long readed = fileIo.transferFrom(ch, meta.offset() + transferred, batchSize); + if (readed == 0) + throw new IOException("Channel is reached the end of stream. Probably, channel is closed on the remote node"); + if (readed > 0) transferred += readed; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 3e1b031cc5b10..c7fbdd68ff4d7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -65,6 +65,7 @@ import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; @@ -265,6 +266,9 @@ public class GridIoManager extends GridManagerAdapter topicTransmissionHnds = new ConcurrentHashMap<>(); @@ -287,6 +291,9 @@ public class GridIoManager extends GridManagerAdapter lsnrMap = new ConcurrentHashMap<>(); @@ -390,6 +397,7 @@ public GridIoManager(GridKernalContext ctx) { ioMetric.register(RCVD_BYTES_CNT, spi::getReceivedBytesCount, "Received bytes count."); retryCnt = ctx.config().getNetworkSendRetryCount(); + netTimeoutMs = (int)ctx.config().getNetworkTimeout(); } /** @@ -1100,14 +1108,16 @@ private void format(StringBuilder b, Collection> pairs, SimpleD stopping = true; - topicTransmissionHnds.clear(); + synchronized (rcvMux) { + topicTransmissionHnds.clear(); - for (ReceiverContext rctx : rcvCtxs.values()) { - interruptRecevier(rctx, new NodeStoppingException("Local node io manager requested to be stopped: " - + ctx.localNodeId())); - } + for (ReceiverContext rctx : rcvCtxs.values()) { + interruptRecevier(rctx, new NodeStoppingException("Local node io manager requested to be stopped: " + + ctx.localNodeId())); + } - rcvCtxs.clear(); + rcvCtxs.clear(); + } } finally { busyLock.writeLock().unlock(); @@ -1872,11 +1882,13 @@ public void addTransmissionHandler(Object topic, TransmissionHandler hnd) { * @param topic The topic to erase handler from. */ public void removeTransmissionHandler(Object topic) { - topicTransmissionHnds.remove(topic); + synchronized (rcvMux) { + topicTransmissionHnds.remove(topic); - interruptRecevier(rcvCtxs.remove(topic), - new IgniteCheckedException("Receiver has been closed due to removing corresponding transmission handler " + - "on local node [nodeId=" + ctx.localNodeId() + ']')); + interruptRecevier(rcvCtxs.remove(topic), + new IgniteCheckedException("Receiver has been closed due to removing corresponding transmission handler " + + "on local node [nodeId=" + ctx.localNodeId() + ']')); + } } /** @@ -2667,7 +2679,7 @@ public int getOutboundMessagesQueueSize() { * @param ctx Receiver context to use. * @param ex Exception to close receiver with. */ - private static void interruptRecevier(ReceiverContext ctx, IgniteCheckedException ex) { + private void interruptRecevier(ReceiverContext ctx, IgniteCheckedException ex) { if (ctx == null) return; @@ -2678,6 +2690,8 @@ private static void interruptRecevier(ReceiverContext ctx, IgniteCheckedExceptio new TransmissionMeta(ex) : ctx.lastState.error(ex); ctx.hnd.onException(ctx.nodeId, ex); + + U.error(log, "Receiver has been interrupted due to an excpetion occurred [ctx=" + ctx + ']', ex); } } @@ -2698,15 +2712,6 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa ", channel=" + ch + ']'); } - TransmissionHandler hnd = topicTransmissionHnds.get(topic); - - if (hnd == null) { - U.warn(log, "There is no handler for a given topic. Channel will be closed [nodeId=" + nodeId + - ", topic=" + topic + ']'); - - return; - } - if (initMsg == null || initMsg.sesId() == null) { U.warn(log, "There is no initial message provied for given topic. Opened channel will be closed " + "[nodeId=" + nodeId + ", topic=" + topic + ", initMsg=" + initMsg + ']'); @@ -2714,66 +2719,75 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa return; } - configureChannel(ctx.config(), ch); + configureChannel(ch, netTimeoutMs); in = new ObjectInputStream(ch.socket().getInputStream()); out = new ObjectOutputStream(ch.socket().getOutputStream()); - if (log.isDebugEnabled()) { - log.debug("Trasmission opens a new channel [nodeId=" + nodeId + ", topic=" + topic + - ", initMsg=" + initMsg + ']'); - } - + IgniteCheckedException err = null; IgniteUuid newSesId = initMsg.sesId(); - synchronized (rcvCtxs) { - rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(nodeId, hnd, newSesId)); + synchronized (rcvMux) { + TransmissionHandler hnd = topicTransmissionHnds.get(topic); - if (rcvCtx.timeoutObj != null) - ctx.timeout().removeTimeoutObject(rcvCtx.timeoutObj); + if (hnd == null) { + U.warn(log, "There is no handler for a given topic. Channel will be closed [nodeId=" + nodeId + + ", topic=" + topic + ']'); - // Do not allow multiple connection for the same session - boolean activated = rcvCtx.active.compareAndSet(false, true); + return; + } - if (!activated && newSesId.equals(rcvCtx.sesId)) { - IOException e = new IOException("Receiver has not completed yet previous data processing. " + - "Wait for the next connection attempt."); + rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(nodeId, hnd, newSesId)); - U.warn(log, e); + // Do not allow multiple connection for the same session + if (!newSesId.equals(rcvCtx.sesId)) { + if (!rcvCtx.active) { + // Attempt to receive file with new session id. Context must be reinited, + // previous session must be failed. + interruptRecevier(rcvCtx, new IgniteCheckedException("Process has been aborted " + + "by transfer attempt with a new session [sesId=" + newSesId + ", nodeId=" + nodeId + ']')); - out.writeObject(new TransmissionMeta(e)); + rcvCtxs.put(topic, rcvCtx = new ReceiverContext(nodeId, hnd, newSesId)); + } + else { + err = new IgniteCheckedException("Requested topic is busy by another transmission. " + + "It's not allowed to process different sessions over the same topic simultaneously. " + + "Channel will be closed [initMsg=" + initMsg + ", channel=" + ch + ", nodeId=" + nodeId + ']'); - return; + U.error(log, err); + } } - else if (!activated && !newSesId.equals(rcvCtx.sesId)) { - IgniteCheckedException ex = new IgniteCheckedException("Receivers topic is busy by another transmission. " + - "It's not allowed to process different sessions over the same topic simultaneously. " + - "Channel will be closed [initMsg=" + initMsg + ", channel=" + ch + ", nodeId=" + nodeId + ']'); + } - U.error(log, ex); + if (err != null) { + out.writeObject(new TransmissionMeta(err)); - out.writeObject(new TransmissionMeta(ex)); + return; + } - return; - } + if (log.isDebugEnabled()) { + log.debug("Trasmission open a new channel [nodeId=" + nodeId + ", topic=" + topic + + ", initMsg=" + initMsg + ']'); + } - if (!newSesId.equals(rcvCtx.sesId)) { - // Attempt to receive file with new session id. Context must be reinited, - // previous session must be failed. - interruptRecevier(rcvCtx, new IgniteCheckedException("Process has been aborted " + - "by transfer attempt with a new session [sesId=" + newSesId + ", nodeId=" + nodeId + ']')); + if (!rcvCtx.lock.tryLock(netTimeoutMs, TimeUnit.MILLISECONDS)) + throw new IgniteException("Wait for the previous receiver finished its work timeouted: " + rcvCtx); - rcvCtx = new ReceiverContext(nodeId, hnd, newSesId); - rcvCtx.active.set(true); + try { + rcvCtx.active = true; - rcvCtxs.put(topic, rcvCtx); - } - } + if (rcvCtx.timeoutObj != null) + ctx.timeout().removeTimeoutObject(rcvCtx.timeoutObj); - // Send previous context state to sync remote and local node (on manager connected). - out.writeObject(rcvCtx.lastState == null ? new TransmissionMeta() : rcvCtx.lastState); + // Send previous context state to sync remote and local node (on manager connected). + out.writeObject(rcvCtx.lastState == null ? new TransmissionMeta() : rcvCtx.lastState); - receiveFromChannel(topic, rcvCtx, in, out, ch); + receiveFromChannel(topic, rcvCtx, in, out, ch); + } + finally { + rcvCtx.active = false; + rcvCtx.lock.unlock(); + } } catch (Throwable t) { U.error(log, "Download session cannot be finished due to an unexpected error [ctx=" + rcvCtx + ']', t); @@ -2781,9 +2795,6 @@ else if (!activated && !newSesId.equals(rcvCtx.sesId)) { interruptRecevier(rcvCtx, new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t)); } finally { - if (rcvCtx != null) - rcvCtx.active.set(false); - U.closeQuiet(in); U.closeQuiet(out); U.closeQuiet(ch); @@ -2870,33 +2881,37 @@ private void receiveFromChannel( throw new IgniteException(e); } catch (IOException e) { - // Waiting for re-establishing connection. - U.warn(log, "Сonnection from the remote node lost. Will wait for the new one to continue file " + - "download " + "[nodeId=" + rcvCtx.nodeId + ", sesKey=" + rcvCtx.sesId + ']', e); - rcvCtx.retries++; - if (rcvCtx.retries == retryCnt) { + if (rcvCtx.retries >= retryCnt) { throw new IgniteException("Number of retry attempts to download file exceeded the limit. " + "Max attempts: " + retryCnt, e); } - ctx.timeout().addTimeoutObject(rcvCtx.timeoutObj = new GridTimeoutObject() { + // Waiting for re-establishing connection. + U.warn(log, "Сonnection from the remote node lost. Will wait for the new one to continue file " + + "download " + "[nodeId=" + rcvCtx.nodeId + ", sesKey=" + rcvCtx.sesId + ']', e); + + long startTs = U.currentTimeMillis(); + + boolean added = ctx.timeout().addTimeoutObject(rcvCtx.timeoutObj = new GridTimeoutObject() { @Override public IgniteUuid timeoutId() { return rcvCtx.sesId; } @Override public long endTime() { - return U.currentTimeMillis() + ctx.config().getNetworkTimeout(); + return startTs + netTimeoutMs; } @Override public void onTimeout() { - ReceiverContext rcvCtx0 = rcvCtxs.remove(topic); + ReceiverContext rcvCtx0 = rcvCtxs.get(topic); interruptRecevier(rcvCtx0, new IgniteCheckedException("Receiver is closed due to " + - "reconnect timeout has been occured")); + "waiting for the reconnect has been timeouted")); } }); + + assert added; } } @@ -2969,13 +2984,13 @@ private TransmissionReceiver createReceiver( } /** - * @param cfg Ignite ocnfiguration to configure channel with. * @param channel Socket channel to configure blocking mode. + * @param timeout Ignite network ocnfiguration timeout. * @throws IOException If fails. */ - private static void configureChannel(IgniteConfiguration cfg, SocketChannel channel) throws IOException { + private static void configureChannel(SocketChannel channel, int timeout) throws IOException { // Timeout must be enabled prior to entering the blocking mode to have effect. - channel.socket().setSoTimeout((int)cfg.getNetworkTimeout()); + channel.socket().setSoTimeout(timeout); channel.configureBlocking(true); } @@ -3013,12 +3028,15 @@ private static class ReceiverContext { /** Unique session request id. */ private final IgniteUuid sesId; - /** Handler currently in use flag. */ - private final AtomicBoolean active = new AtomicBoolean(); - /** Flag indicates that current file handling process must be interrupted. */ private final AtomicBoolean interrupted = new AtomicBoolean(); + /** Only one thread can handle receiver context. */ + private final Lock lock = new ReentrantLock(); + + /** Handler currently in use flag. */ + private volatile boolean active; + /** Flag indicates session started. */ private boolean sesStarted; @@ -3159,7 +3177,7 @@ private TransmissionMeta connect() throws IgniteCheckedException, IOException { new SessionChannelMessage(sesKey.get2())) .get(); - configureChannel(ctx.config(), channel); + configureChannel(channel, netTimeoutMs); this.channel = (WritableByteChannel)channel; out = new ObjectOutputStream(channel.socket().getOutputStream()); @@ -3272,7 +3290,7 @@ public void send( retries++; - if (retries == retryCnt) { + if (retries >= retryCnt) { throw new IgniteException("The number of retry attempts to upload file exceeded " + "the limit: " + retryCnt, e); } @@ -3309,12 +3327,16 @@ public void send( try { senderStopFlags.remove(sesKey); - if (out != null) { - U.log(log, "Close file writer session: " + sesKey); + ObjectOutput out0 = out; - // Send transmission close flag. - out.writeBoolean(true); - } + if (out0 == null) + return; + + U.log(log, "Close file writer session: " + sesKey); + + // Send transmission close flag. + out0.writeBoolean(true); + out0.flush(); } catch (IOException e) { U.warn(log, "An excpetion while writing close session flag occured. " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java index 7efcea9c8a8ab..0aaff79388674 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java @@ -227,8 +227,8 @@ private class TimeoutWorker extends GridWorker { try { boolean rmvd = timeoutObjs.remove(timeoutObj); - if (log.isDebugEnabled()) - log.debug("Timeout has occurred [obj=" + timeoutObj + ", process=" + rmvd + ']'); + if (log.isInfoEnabled()) + log.info("Timeout has occurred [obj=" + timeoutObj + ", process=" + rmvd + ']'); if (rmvd) timeoutObj.onTimeout(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index c53328bf2a931..f76f9ddc999e5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -24,6 +24,7 @@ import java.io.Serializable; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.channels.Channel; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; import java.nio.file.OpenOption; @@ -35,18 +36,21 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; import java.util.function.Consumer; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.IgniteException; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator; @@ -55,6 +59,9 @@ import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.After; @@ -72,6 +79,9 @@ public class GridIoManagerFileTransmissionSelfTest extends GridCommonAbstractTes /** Number of cache keys to generate. */ private static final long CACHE_SIZE = 50_000L; + /** Network timeout in ms. */ + private static final long NET_TIMEOUT_MS = 2000L; + /** Temporary directory to store files. */ private static final String TEMP_FILES_DIR = "ctmp"; @@ -120,7 +130,6 @@ public void before() throws Exception { @After public void after() { stopAllGrids(); - U.closeQuiet(fileIo[0]); } @@ -131,7 +140,9 @@ public void after() { .setDefaultDataRegionConfiguration(new DataRegionConfiguration() .setPersistenceEnabled(true) .setMaxSize(500L * 1024 * 1024))) - .setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME)); + .setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME)) + .setCommunicationSpi(new BlockingOpenChannelCommunicationSpi()) + .setNetworkTimeout(NET_TIMEOUT_MS); } /** @@ -319,6 +330,70 @@ public void testFileHandlerOnReceiverLeft() throws Exception { } } + /** + * @throws Exception If fails. + */ + @Test + public void tesFileHandlerTimeouted() throws Exception { + IgniteEx rcv = startGrid(1); + IgniteEx snd = startGrid(0); + + final AtomicInteger chunksCnt = new AtomicInteger(); + final CountDownLatch sndLatch = ((BlockingOpenChannelCommunicationSpi)snd.context() + .config() + .getCommunicationSpi()).latch; + final AtomicReference refErr = new AtomicReference<>(); + + snd.cluster().active(true); + + File fileToSend = createFileRandomData("testFile", 5 * 1024 * 1024); + + snd.context().io().transfererFileIoFactory(new FileIOFactory() { + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO fileIo = IO_FACTORY.create(file, modes); + + return new FileIODecorator(fileIo) { + /** {@inheritDoc} */ + @Override public long transferTo(long position, long count, WritableByteChannel target) + throws IOException { + if (chunksCnt.incrementAndGet() == 10) { + target.close(); + + ((BlockingOpenChannelCommunicationSpi)snd.context() + .config() + .getCommunicationSpi()).block = true; + } + + return super.transferTo(position, count, target); + } + }; + } + }); + + rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore) { + @Override public void onException(UUID nodeId, Throwable err) { + refErr.compareAndSet(null, err); + + sndLatch.countDown(); + } + }); + + try (GridIoManager.TransmissionSender sender = snd.context() + .io() + .openTransmissionSender(rcv.localNode().id(), topic)) { + sender.send(fileToSend, TransmissionPolicy.FILE); + } + catch (IgniteCheckedException | IOException | InterruptedException e) { + // Ignore err + U.warn(log, e); + } + + assertNotNull("Timeout exception not occurred", refErr.get()); + assertEquals("Type of timeout excpetion incorrect: " + refErr.get(), + IgniteCheckedException.class, + refErr.get().getClass()); + } + /** * @throws Exception If fails. */ @@ -835,6 +910,32 @@ public DefaultTransmissionHandler(IgniteEx rcv, File fileToSend, File tempStorag } } + /** */ + private static class BlockingOpenChannelCommunicationSpi extends TcpCommunicationSpi { + /** Latch to wait at. */ + private final CountDownLatch latch = new CountDownLatch(1); + + /** {@code true} to start waiting. */ + private volatile boolean block; + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture openChannel(ClusterNode remote, + Message initMsg) throws IgniteSpiException { + try { + if (block) { + U.log(log, "Start waiting on trying open a new channel"); + + latch.await(5, TimeUnit.SECONDS); + } + } + catch (InterruptedException e) { + throw new IgniteException(e); + } + + return super.openChannel(remote, initMsg); + } + } + /** * The defailt implementation of transmit session. */ From 435967d19ef645ba337997bd2d0f72b638b840a5 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 17:42:03 +0300 Subject: [PATCH 46/69] IGNITE-10619: set transmission factory via reflection --- .../managers/communication/GridIoManager.java | 10 ---------- ...GridIoManagerFileTransmissionSelfTest.java | 19 ++++++++++++++----- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index c7fbdd68ff4d7..e00bc939b84c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2933,16 +2933,6 @@ private void validate(TransmissionMeta prev, TransmissionMeta next) { " different transmission policy [prev=" + prev + ", next=" + next + ']'); } - /** - * Set factory to produce an FileIO abstraction over sended\received files. - * @see #fileIoFactory - * - * @param factory A new factory instance for creating {@link FileIO} - */ - void transfererFileIoFactory(FileIOFactory factory) { - fileIoFactory = factory; - } - /** * @param nodeId Remote node id. * @param hnd Currnet handler instance which produces file handlers. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index f76f9ddc999e5..2c2a8a540224e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -71,6 +71,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX; import static org.apache.ignite.internal.util.IgniteUtils.fileCount; +import static org.apache.ignite.testframework.GridTestUtils.setFieldValue; /** * Test file transmission mamanger operations. @@ -302,7 +303,7 @@ public void testFileHandlerOnReceiverLeft() throws Exception { File fileToSend = createFileRandomData("testFile", fileSizeBytes); - snd.context().io().transfererFileIoFactory(new FileIOFactory() { + transmissionFileIoFactory(snd, new FileIOFactory() { @Override public FileIO create(File file, OpenOption... modes) throws IOException { FileIO fileIo = IO_FACTORY.create(file, modes); @@ -348,7 +349,7 @@ public void tesFileHandlerTimeouted() throws Exception { File fileToSend = createFileRandomData("testFile", 5 * 1024 * 1024); - snd.context().io().transfererFileIoFactory(new FileIOFactory() { + transmissionFileIoFactory(snd, new FileIOFactory() { @Override public FileIO create(File file, OpenOption... modes) throws IOException { FileIO fileIo = IO_FACTORY.create(file, modes); @@ -407,7 +408,7 @@ public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { File fileToSend = createFileRandomData("tempFile15Mb", 15 * 1024 * 1024); File downloadTo = U.resolveWorkDirectory(tempStore.getAbsolutePath(), "download", true); - snd.context().io().transfererFileIoFactory(new FileIOFactory() { + transmissionFileIoFactory(snd, new FileIOFactory() { @Override public FileIO create(File file, OpenOption... modes) throws IOException { FileIO fileIo = IO_FACTORY.create(file, modes); @@ -466,7 +467,7 @@ public void testFileHandlerReconnectOnReadFail() throws Exception { File fileToSend = createFileRandomData("testFile", 5 * 1024 * 1024); final AtomicInteger readedChunks = new AtomicInteger(); - rcv.context().io().transfererFileIoFactory(new FileIOFactory() { + transmissionFileIoFactory(rcv, new FileIOFactory() { @Override public FileIO create(File file, OpenOption... modes) throws IOException { fileIo[0] = IO_FACTORY.create(file, modes); @@ -693,7 +694,7 @@ public void testChunkHandlerWithReconnect() throws Exception { File fileToSend = createFileRandomData(filePrefix, 10 * 1024 * 1024); - snd.context().io().transfererFileIoFactory(new FileIOFactory() { + transmissionFileIoFactory(snd, new FileIOFactory() { @Override public FileIO create(File file, OpenOption... modes) throws IOException { FileIO fileIo = IO_FACTORY.create(file, modes); @@ -857,6 +858,14 @@ private File createFileRandomData(String name, final int size) throws IOExceptio return out; } + /** + * @param ignite Ignite instance to set factory. + * @param factory New factory to use. + */ + private static void transmissionFileIoFactory(IgniteEx ignite, FileIOFactory factory) { + setFieldValue(ignite.context().io(), "fileIoFactory", factory); + } + /** * @param fileToSend Source file to check CRC. * @param fileReceived Destination file to check CRC. From 155eadb6d6544804c804c19cd272058cc63ecb6d Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 17:45:14 +0300 Subject: [PATCH 47/69] IGNITE-10619: revert timeout processor --- .../internal/processors/timeout/GridTimeoutProcessor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java index 0aaff79388674..7efcea9c8a8ab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java @@ -227,8 +227,8 @@ private class TimeoutWorker extends GridWorker { try { boolean rmvd = timeoutObjs.remove(timeoutObj); - if (log.isInfoEnabled()) - log.info("Timeout has occurred [obj=" + timeoutObj + ", process=" + rmvd + ']'); + if (log.isDebugEnabled()) + log.debug("Timeout has occurred [obj=" + timeoutObj + ", process=" + rmvd + ']'); if (rmvd) timeoutObj.onTimeout(); From 5ed49508554f572e262c35fc5d533c95e729ba43 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 18:22:29 +0300 Subject: [PATCH 48/69] IGNITE-10619: remove ctx on timeout, interrupt outside mutex --- .../managers/communication/FileSender.java | 5 ++- .../managers/communication/GridIoManager.java | 45 ++++++++++--------- ...GridIoManagerFileTransmissionSelfTest.java | 2 +- 3 files changed, 28 insertions(+), 24 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 1bc884d8ec04d..9fc5d29766e30 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -129,8 +129,11 @@ private void updateSenderState(TransmissionMeta rcvMeta) { assert fileIo != null; // The remote node doesn't have a file meta info. - if (rcvMeta == null || rcvMeta.offset() < 0) + if (rcvMeta == null || rcvMeta.offset() < 0) { + transferred = 0; + return; + } long uploadedBytes = rcvMeta.offset() - meta.offset(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index e00bc939b84c5..7ae23080f880b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -41,11 +41,13 @@ import java.util.Date; import java.util.Deque; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Queue; +import java.util.Set; import java.util.UUID; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.ConcurrentHashMap; @@ -1108,16 +1110,20 @@ private void format(StringBuilder b, Collection> pairs, SimpleD stopping = true; + Set rcvs; + synchronized (rcvMux) { topicTransmissionHnds.clear(); - for (ReceiverContext rctx : rcvCtxs.values()) { - interruptRecevier(rctx, new NodeStoppingException("Local node io manager requested to be stopped: " - + ctx.localNodeId())); - } + rcvs = new HashSet<>(rcvCtxs.values()); rcvCtxs.clear(); } + + for (ReceiverContext rctx : rcvs) { + interruptRecevier(rctx, new NodeStoppingException("Local node io manager requested to be stopped: " + + ctx.localNodeId())); + } } finally { busyLock.writeLock().unlock(); @@ -2726,6 +2732,7 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa IgniteCheckedException err = null; IgniteUuid newSesId = initMsg.sesId(); + ReceiverContext prevRcvCtx = null; synchronized (rcvMux) { TransmissionHandler hnd = topicTransmissionHnds.get(topic); @@ -2741,24 +2748,25 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa // Do not allow multiple connection for the same session if (!newSesId.equals(rcvCtx.sesId)) { - if (!rcvCtx.active) { - // Attempt to receive file with new session id. Context must be reinited, - // previous session must be failed. - interruptRecevier(rcvCtx, new IgniteCheckedException("Process has been aborted " + - "by transfer attempt with a new session [sesId=" + newSesId + ", nodeId=" + nodeId + ']')); - - rcvCtxs.put(topic, rcvCtx = new ReceiverContext(nodeId, hnd, newSesId)); - } - else { + if (rcvCtx.active) { err = new IgniteCheckedException("Requested topic is busy by another transmission. " + "It's not allowed to process different sessions over the same topic simultaneously. " + "Channel will be closed [initMsg=" + initMsg + ", channel=" + ch + ", nodeId=" + nodeId + ']'); U.error(log, err); } + else { + // Attempt to receive file with new session id. Context must be reinited, + // previous session must be closed. + prevRcvCtx = rcvCtxs.put(topic, rcvCtx = new ReceiverContext(nodeId, hnd, newSesId)); + } } } + // Stop previous context if exist + interruptRecevier(prevRcvCtx, new IgniteCheckedException("Process has been aborted " + + "by transfer attempt with a new session [sesId=" + newSesId + ", nodeId=" + nodeId + ']')); + if (err != null) { out.writeObject(new TransmissionMeta(err)); @@ -2881,16 +2889,9 @@ private void receiveFromChannel( throw new IgniteException(e); } catch (IOException e) { - rcvCtx.retries++; - - if (rcvCtx.retries >= retryCnt) { - throw new IgniteException("Number of retry attempts to download file exceeded the limit. " + - "Max attempts: " + retryCnt, e); - } - // Waiting for re-establishing connection. U.warn(log, "Сonnection from the remote node lost. Will wait for the new one to continue file " + - "download " + "[nodeId=" + rcvCtx.nodeId + ", sesKey=" + rcvCtx.sesId + ']', e); + "receive [nodeId=" + rcvCtx.nodeId + ", sesKey=" + rcvCtx.sesId + ']', e); long startTs = U.currentTimeMillis(); @@ -2904,7 +2905,7 @@ private void receiveFromChannel( } @Override public void onTimeout() { - ReceiverContext rcvCtx0 = rcvCtxs.get(topic); + ReceiverContext rcvCtx0 = rcvCtxs.remove(topic); interruptRecevier(rcvCtx0, new IgniteCheckedException("Receiver is closed due to " + "waiting for the reconnect has been timeouted")); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 2c2a8a540224e..374a6ea2fcaaf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -335,7 +335,7 @@ public void testFileHandlerOnReceiverLeft() throws Exception { * @throws Exception If fails. */ @Test - public void tesFileHandlerTimeouted() throws Exception { + public void tesFileHandlerReconnectTimeouted() throws Exception { IgniteEx rcv = startGrid(1); IgniteEx snd = startGrid(0); From 76e04c26b6e7d50a6ea2572f0e10956074a06f89 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 20:13:30 +0300 Subject: [PATCH 49/69] IGNITE-10619: new session will get an error is try interrupt an active one --- .../managers/communication/GridIoManager.java | 43 ++++++------------- ...GridIoManagerFileTransmissionSelfTest.java | 21 +++++---- 2 files changed, 25 insertions(+), 39 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 7ae23080f880b..4aad21f67df5c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2685,7 +2685,7 @@ public int getOutboundMessagesQueueSize() { * @param ctx Receiver context to use. * @param ex Exception to close receiver with. */ - private void interruptRecevier(ReceiverContext ctx, IgniteCheckedException ex) { + private void interruptRecevier(ReceiverContext ctx, Exception ex) { if (ctx == null) return; @@ -2732,7 +2732,6 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa IgniteCheckedException err = null; IgniteUuid newSesId = initMsg.sesId(); - ReceiverContext prevRcvCtx = null; synchronized (rcvMux) { TransmissionHandler hnd = topicTransmissionHnds.get(topic); @@ -2748,25 +2747,14 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa // Do not allow multiple connection for the same session if (!newSesId.equals(rcvCtx.sesId)) { - if (rcvCtx.active) { - err = new IgniteCheckedException("Requested topic is busy by another transmission. " + - "It's not allowed to process different sessions over the same topic simultaneously. " + - "Channel will be closed [initMsg=" + initMsg + ", channel=" + ch + ", nodeId=" + nodeId + ']'); + err = new IgniteCheckedException("Requested topic is busy by another transmission. " + + "It's not allowed to process different sessions over the same topic simultaneously. " + + "Channel will be closed [initMsg=" + initMsg + ", channel=" + ch + ", nodeId=" + nodeId + ']'); - U.error(log, err); - } - else { - // Attempt to receive file with new session id. Context must be reinited, - // previous session must be closed. - prevRcvCtx = rcvCtxs.put(topic, rcvCtx = new ReceiverContext(nodeId, hnd, newSesId)); - } + U.error(log, err); } } - // Stop previous context if exist - interruptRecevier(prevRcvCtx, new IgniteCheckedException("Process has been aborted " + - "by transfer attempt with a new session [sesId=" + newSesId + ", nodeId=" + nodeId + ']')); - if (err != null) { out.writeObject(new TransmissionMeta(err)); @@ -2782,24 +2770,25 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa throw new IgniteException("Wait for the previous receiver finished its work timeouted: " + rcvCtx); try { - rcvCtx.active = true; - if (rcvCtx.timeoutObj != null) ctx.timeout().removeTimeoutObject(rcvCtx.timeoutObj); // Send previous context state to sync remote and local node (on manager connected). out.writeObject(rcvCtx.lastState == null ? new TransmissionMeta() : rcvCtx.lastState); - receiveFromChannel(topic, rcvCtx, in, out, ch); + if (rcvCtx.lastState == null || rcvCtx.lastState.error() == null) + receiveFromChannel(topic, rcvCtx, in, out, ch); + else + interruptRecevier(rcvCtxs.remove(topic), rcvCtx.lastState.error()); } finally { - rcvCtx.active = false; rcvCtx.lock.unlock(); } } catch (Throwable t) { U.error(log, "Download session cannot be finished due to an unexpected error [ctx=" + rcvCtx + ']', t); + // Do not remove receiver context here, since sender will recconect to get this error interruptRecevier(rcvCtx, new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t)); } finally { @@ -2874,7 +2863,7 @@ private void receiveFromChannel( U.log(log, "File has been received " + "[name=" + rcvCtx.rcv.state().name() + ", transferred=" + rcvCtx.rcv.transferred() + ", time=" + (double)((U.currentTimeMillis() - startTime) / 1000) + " sec" + - ", retries=" + rcvCtx.retries + ", remoteId=" + rcvCtx.nodeId + ']'); + ", remoteId=" + rcvCtx.nodeId + ']'); rcvCtx.rcv = null; } @@ -2905,9 +2894,7 @@ private void receiveFromChannel( } @Override public void onTimeout() { - ReceiverContext rcvCtx0 = rcvCtxs.remove(topic); - - interruptRecevier(rcvCtx0, new IgniteCheckedException("Receiver is closed due to " + + interruptRecevier(rcvCtxs.remove(topic), new IgniteCheckedException("Receiver is closed due to " + "waiting for the reconnect has been timeouted")); } }); @@ -3025,15 +3012,9 @@ private static class ReceiverContext { /** Only one thread can handle receiver context. */ private final Lock lock = new ReentrantLock(); - /** Handler currently in use flag. */ - private volatile boolean active; - /** Flag indicates session started. */ private boolean sesStarted; - /** The number of retry attempts of current session to wait. */ - private int retries; - /** Last infinished downloading object. */ private TransmissionReceiver rcv; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 374a6ea2fcaaf..9c4cba2bf99d4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -585,22 +585,27 @@ public void testFileHandlerNextWriterOpened() throws Exception { assertEquals(fileToSend.length(), rcvFile.length()); assertCrcEquals(fileToSend, rcvFile); + } + + /** + * @throws Exception If fails. + */ + @Test(expected = IgniteException.class) + public void testFileHandlerSendToNullTopic() throws Exception { + IgniteEx snd = startGrid(0); + IgniteEx rcv = startGrid(1); - // Remove topic handler and fail + snd.cluster().active(true); + + // Ensure topic handler is empty rcv.context().io().removeTransmissionHandler(topic); // Open next writer on removed topic. try (GridIoManager.TransmissionSender sender = snd.context() .io() .openTransmissionSender(rcv.localNode().id(), topic)) { - sender.send(fileToSend, TransmissionPolicy.FILE); + sender.send(createFileRandomData("File_1MB", 1024 * 1024), TransmissionPolicy.FILE); } - catch (IgniteException e) { - // Must catch execption here. - expectedErr = e; - } - - assertNotNull("Transmission must ends with an exception", expectedErr); } /** From d968a567e21c2ac070fb216a4a01502b52e74a49 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 20:21:53 +0300 Subject: [PATCH 50/69] IGNITE-10619: remove onBegin, onEnd handler methods --- .../managers/communication/GridIoManager.java | 12 ------ .../communication/TransmissionHandler.java | 10 ----- ...GridIoManagerFileTransmissionSelfTest.java | 39 ++----------------- 3 files changed, 4 insertions(+), 57 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 4aad21f67df5c..e2e0be40bbccd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2811,13 +2811,6 @@ private void receiveFromChannel( ObjectOutputStream out, ReadableByteChannel ch ) throws NodeStoppingException, InterruptedException { - // Begin method must be called only once. - if (!rcvCtx.sesStarted) { - rcvCtx.hnd.onBegin(rcvCtx.nodeId); - - rcvCtx.sesStarted = true; - } - try { while (true) { if (Thread.currentThread().isInterrupted()) @@ -2829,8 +2822,6 @@ private void receiveFromChannel( boolean exit = in.readBoolean(); if (exit) { - rcvCtx.hnd.onEnd(rcvCtx.nodeId); - rcvCtxs.remove(topic); break; @@ -3012,9 +3003,6 @@ private static class ReceiverContext { /** Only one thread can handle receiver context. */ private final Lock lock = new ReentrantLock(); - /** Flag indicates session started. */ - private boolean sesStarted; - /** Last infinished downloading object. */ private TransmissionReceiver rcv; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java index 1e0d1748606a2..8749f18f53821 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java @@ -40,16 +40,6 @@ * {@link TransmissionPolicy#CHUNK} the ChunkHandler will be picked up. */ public interface TransmissionHandler { - /** - * @param nodeId The remote node id receive request for transmission from. - */ - public void onBegin(UUID nodeId); - - /** - * The end of session transmission process. - */ - public void onEnd(UUID nodeId); - /** * @param err The err of fail handling process. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 9c4cba2bf99d4..75d0287572ad2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -169,10 +169,6 @@ public void testFileHandlerBase() throws Exception { assertTrue(snd.context().io().fileTransmissionSupported(rcv.localNode())); rcv.context().io().addTransmissionHandler(topic, new TransmissionHandlerAdapter() { - @Override public void onBegin(UUID nodeId) { - assertEquals(snd.localNode().id(), nodeId); - } - @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { return new File(tempStore, fileMeta.name()).getAbsolutePath(); } @@ -237,7 +233,7 @@ public void testFileHandlerBase() throws Exception { * @throws Exception If fails. */ @Test(expected = IgniteCheckedException.class) - public void testFileHandlerOnBeginFails() throws Exception { + public void testFileHandlerFilePathThrowsEx() throws Exception { final String exTestMessage = "Test exception. Handler initialization failed at onBegin."; IgniteEx snd = startGrid(0); @@ -248,14 +244,8 @@ public void testFileHandlerOnBeginFails() throws Exception { File fileToSend = createFileRandomData("1Mb", 1024 * 1024); rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore) { - @Override public void onBegin(UUID nodeId) { - throw new IgniteException(exTestMessage); - } - @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { - fail("filePath must never be called"); - - return null; + throw new IgniteException(exTestMessage); } @Override public Consumer fileHandler(UUID nodeId, TransmissionMeta initMeta) { @@ -270,12 +260,6 @@ public void testFileHandlerOnBeginFails() throws Exception { return super.chunkHandler(nodeId, initMeta); } - @Override public void onEnd(UUID nodeId) { - fail("onEnd must never be called"); - - super.onEnd(nodeId); - } - @Override public void onException(UUID nodeId, Throwable err) { assertEquals(exTestMessage, err.getMessage()); } @@ -625,15 +609,10 @@ public void testFileHandlerChannelCloseIfAnotherOpened() throws Exception { File fileToSend = createFileRandomData("file5MBSize", fileSizeBytes); rcv.context().io().addTransmissionHandler(topic, new DefaultTransmissionHandler(rcv, fileToSend, tempStore) { - @Override public void onBegin(UUID nodeId) { + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { waitLatch.countDown(); - try { - waitLatch.await(5, TimeUnit.SECONDS); - } - catch (InterruptedException e) { - throw new IgniteException(e); - } + return super.filePath(nodeId, fileMeta); } }); @@ -954,11 +933,6 @@ private static class BlockingOpenChannelCommunicationSpi extends TcpCommunicatio * The defailt implementation of transmit session. */ private static class TransmissionHandlerAdapter implements TransmissionHandler { - /** {@inheritDoc} */ - @Override public void onBegin(UUID nodeId) { - // No-op. - } - /** {@inheritDoc} */ @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { return null; @@ -974,11 +948,6 @@ private static class TransmissionHandlerAdapter implements TransmissionHandler { return null; } - /** {@inheritDoc} */ - @Override public void onEnd(UUID nodeId) { - // No-op. - } - /** {@inheritDoc} */ @Override public void onException(UUID nodeId, Throwable err) { // No-op. From 1d37e4ea3e64b02abd7b21bb8ad6d580ee65a597 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 20:26:45 +0300 Subject: [PATCH 51/69] IGNITE-10619: remove null from close --- .../ignite/internal/managers/communication/ChunkReceiver.java | 2 +- .../ignite/internal/managers/communication/FileReceiver.java | 2 -- .../ignite/internal/managers/communication/FileSender.java | 4 ---- 3 files changed, 1 insertion(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java index 2d19966c928fc..967aee838f32e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java @@ -104,7 +104,7 @@ public ChunkReceiver( /** {@inheritDoc} */ @Override public void close() { - buf = null; + // No-op. } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index fc3b8b37931de..b2de357aba048 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -112,8 +112,6 @@ public FileReceiver( @Override public void close() { U.closeQuiet(fileIo); - fileIo = null; - try { if (transferred != meta.count()) Files.delete(file.toPath()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 9fc5d29766e30..5e9548d257489 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -126,8 +126,6 @@ public void send(WritableByteChannel ch, * @param rcvMeta Conneciton meta info. */ private void updateSenderState(TransmissionMeta rcvMeta) { - assert fileIo != null; - // The remote node doesn't have a file meta info. if (rcvMeta == null || rcvMeta.offset() < 0) { transferred = 0; @@ -168,8 +166,6 @@ private void writeChunk(WritableByteChannel ch) throws IOException { /** {@inheritDoc} */ @Override public void close() { U.closeQuiet(fileIo); - - fileIo = null; } /** {@inheritDoc} */ From 8399aba24bc46b5031f73a5e4531fb9ccb316441 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 21:06:44 +0300 Subject: [PATCH 52/69] IGNITE-10619: minor code changes --- .../managers/communication/GridIoManager.java | 55 ++++++++++--------- 1 file changed, 29 insertions(+), 26 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index e2e0be40bbccd..c97bd573c15a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -288,7 +288,7 @@ public class GridIoManager extends GridManagerAdapter Date: Tue, 6 Aug 2019 21:20:59 +0300 Subject: [PATCH 53/69] IGNITE-10619: add mutex on node left --- .../managers/communication/GridIoManager.java | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index c97bd573c15a8..33e696025250a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -930,18 +930,20 @@ private void format(StringBuilder b, Collection> pairs, SimpleD writeSesEntry.getValue().set(true); } - // Clear the context on the uploader node left. - Iterator> it = rcvCtxs.entrySet().iterator(); + synchronized (rcvMux) { + // Clear the context on the uploader node left. + Iterator> it = rcvCtxs.entrySet().iterator(); - while(it.hasNext()) { - Map.Entry e = it.next(); + while (it.hasNext()) { + Map.Entry e = it.next(); - if (nodeId.equals(e.getValue().nodeId)) { - it.remove(); + if (nodeId.equals(e.getValue().nodeId)) { + it.remove(); - interruptRecevier(e.getValue(), - new ClusterTopologyCheckedException("Remove node left the grid. " + - "Receiver has been stopped : " + nodeId)); + interruptRecevier(e.getValue(), + new ClusterTopologyCheckedException("Remove node left the grid. " + + "Receiver has been stopped : " + nodeId)); + } } } } From 06432858a7345b9c33c4c5cc13c2c0adf8ff44e4 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 6 Aug 2019 21:27:42 +0300 Subject: [PATCH 54/69] IGNITE-10619: minor code test fixes --- .../GridIoManagerFileTransmissionSelfTest.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 75d0287572ad2..77f3302558f48 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -206,15 +206,13 @@ public void testFileHandlerBase() throws Exception { } } - log.info("Writing test cacheParts finished. All Ignite instances will be stopped."); - stopAllGrids(); assertEquals(fileSizes.size(), tempStore.listFiles(fileBinFilter).length); for (File file : cacheParts) { // Check received file lenghs - assertEquals("Received file lenght is incorrect: " + file.getName(), + assertEquals("Received the file length is incorrect: " + file.getName(), fileSizes.get(file.getName()), new Long(file.length())); // Check received params @@ -798,12 +796,8 @@ private void addCacheData(Ignite ignite, String cacheName) { try (IgniteDataStreamer dataStreamer = ignite.dataStreamer(cacheName)) { dataStreamer.allowOverwrite(true); - for (int i = 0; i < CACHE_SIZE; i++) { - if ((i + 1) % (CACHE_SIZE / 10) == 0) - log.info("Prepared " + (i + 1) * 100 / (CACHE_SIZE) + "% entries."); - + for (int i = 0; i < CACHE_SIZE; i++) dataStreamer.addData(i, i + cacheName.hashCode()); - } } } From 150d448518f7d8a0eeda6191677b5ad92f2bb371 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Wed, 7 Aug 2019 14:06:53 +0300 Subject: [PATCH 55/69] IGNITE-10619: change channel creation request future --- .../managers/communication/GridIoManager.java | 55 ++++++++----- .../tcp/TcpCommunicationSpi.java | 77 ++++++++----------- ...GridIoManagerFileTransmissionSelfTest.java | 5 +- 3 files changed, 70 insertions(+), 67 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 33e696025250a..08fc415713e4c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2859,7 +2859,7 @@ private void receiveFromChannel( U.log(log, "File has been received " + "[name=" + rcvCtx.rcv.state().name() + ", transferred=" + rcvCtx.rcv.transferred() + ", time=" + (double)((U.currentTimeMillis() - startTime) / 1000) + " sec" + - ", remoteId=" + rcvCtx.nodeId + ']'); + ", rmtId=" + rcvCtx.nodeId + ']'); rcvCtx.rcv = null; } @@ -3099,7 +3099,7 @@ public ReceiverContext(UUID nodeId, TransmissionHandler hnd, IgniteUuid sesId) { */ public class TransmissionSender implements Closeable { /** Remote node id to connect to. */ - private final UUID remoteId; + private final UUID rmtId; /** Remote topic to connect to. */ private final Object topic; @@ -3117,16 +3117,16 @@ public class TransmissionSender implements Closeable { private ObjectInput in; /** - * @param remoteId The remote note to connect to. + * @param rmtId The remote note to connect to. * @param topic The remote topic to connect to. */ public TransmissionSender( - UUID remoteId, + UUID rmtId, Object topic ) { - this.remoteId = remoteId; + this.rmtId = rmtId; this.topic = topic; - sesKey = new T2<>(remoteId, IgniteUuid.randomUuid()); + sesKey = new T2<>(rmtId, IgniteUuid.randomUuid()); } /** @@ -3135,9 +3135,7 @@ public TransmissionSender( * @throws IOException If fails. */ private TransmissionMeta connect() throws IgniteCheckedException, IOException { - senderStopFlags.putIfAbsent(sesKey, new AtomicBoolean()); - - SocketChannel channel = (SocketChannel)openChannel(remoteId, + SocketChannel channel = (SocketChannel)openChannel(rmtId, topic, new SessionChannelMessage(sesKey.get2())) .get(); @@ -3202,6 +3200,11 @@ public void send( Map params, TransmissionPolicy plc ) throws IgniteCheckedException, InterruptedException, IOException { + long startTime = U.currentTimeMillis(); + int retries = 0; + + senderStopFlags.putIfAbsent(sesKey, new AtomicBoolean()); + try (FileSender snd = new FileSender(file, offset, cnt, @@ -3214,12 +3217,9 @@ public void send( ) { if (log.isDebugEnabled()) { log.debug("Start writing file to remote node [file=" + file.getName() + - ", rmtNodeId=" + remoteId + ", topic=" + topic + ']'); + ", rmtNodeId=" + rmtId + ", topic=" + topic + ']'); } - long startTime = U.currentTimeMillis(); - int retries = 0; - while (true) { if (Thread.currentThread().isInterrupted()) throw new InterruptedException("The thread has been interrupted. Stop uploading file."); @@ -3262,7 +3262,7 @@ public void send( // Re-establish the new connection to continue upload. U.warn(log, "Connection lost while writing a file to remote node and " + - "will be reestablished [remoteId=" + remoteId + ", file=" + file.getName() + + "will be reestablished [rmtId=" + rmtId + ", file=" + file.getName() + ", sesKey=" + sesKey + ", retries=" + retries + ", transferred=" + snd.transferred() + ", total=" + snd.state().count() + ']', e); @@ -3271,19 +3271,36 @@ public void send( U.log(log, "File has been sent to remote node [name=" + file.getName() + ", uploadTime=" + (double)((U.currentTimeMillis() - startTime) / 1000) + " sec, retries=" + retries + - ", transferred=" + snd.transferred() + ", remoteId=" + remoteId +']'); + ", transferred=" + snd.transferred() + ", rmtId=" + rmtId +']'); + + } + catch (IgniteException e) { + closeChannelQuiet(); + if (stopping) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)"); + + if (senderStopFlags.get(sesKey).get()) + throw new ClusterTopologyCheckedException("Remote node left the cluster: " + rmtId, e); + + throw e; } - catch (IgniteCheckedException | InterruptedException e) { + catch (InterruptedException e) { closeChannelQuiet(); throw e; } - catch (Throwable t) { + catch (IgniteCheckedException e) { + closeChannelQuiet(); + + throw new IgniteCheckedException("Excpetion while sending file [rmtId=" + rmtId + + ", file=" + file.getName() + ", sesKey=" + sesKey + ", retries=" + retries + ']', e); + } + catch (Throwable e) { closeChannelQuiet(); - throw new IgniteException("Exception while sending file to the remote node. The process stopped " + - "[remoteId=" + remoteId + ", file=" + file.getName() + ", sesKey=" + sesKey + ']', t); + throw new IgniteException("Unexpected exception while sending file to the remote node. The process stopped " + + "[rmtId=" + rmtId + ", file=" + file.getName() + ", sesKey=" + sesKey + ']', e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 77e605616ab8a..642df530b0d0c 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -43,7 +43,6 @@ import java.util.StringJoiner; import java.util.UUID; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; @@ -360,6 +359,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati /** Message tracker meta for session. */ private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey(); + /** Session future of channel creation request. */ + private static final int CHANNEL_FUT_META = GridNioSessionMetaKey.nextUniqueKey(); + /** * Default local port range (value is 100). * See {@link #setLocalPortRange(int)} for details. @@ -753,18 +755,11 @@ private void closeStaleConnections(ConnectionKey connKey) { } } - private void handleChannelCreateResponse(GridSelectorNioSessionImpl ses, ConnectionKey connKey) { - GridFutureAdapter reqFut = channelReqs.remove(connKey); - - if (reqFut == null) { - U.error(log, "There is not corresponding channel request to the received channel create " + - "response message. Message will be ignored [remoteId=" + connKey.nodeId() + - ", idx=" + connKey.connectionIndex() + ']'); - - ses.close(); - - return; - } + private void handleChannelCreateResponse( + GridSelectorNioSessionImpl ses, + GridFutureAdapter reqFut + ) { + assert reqFut != null; ses.closeSocketOnSessionClose(false); @@ -898,6 +893,22 @@ else if (connKey.dummy()) { metricsLsnr.onMessageReceived(msg, connKey.nodeId()); + if (ses.meta(CHANNEL_FUT_META) != null) { + //Response message received and will be ignored + assert msg instanceof ChannelCreateResponse; + + handleChannelCreateResponse((GridSelectorNioSessionImpl)ses, ses.meta(CHANNEL_FUT_META)); + + return; + } + + if (msg instanceof ChannelCreateRequest) { + handleChannelCreateRequest((GridSelectorNioSessionImpl)ses, connKey, + (ChannelCreateRequest)msg); + + return; + } + IgniteRunnable c; if (msgQueueLimit > 0) { @@ -917,22 +928,7 @@ else if (connKey.dummy()) { else c = NOOP; - if (msg instanceof ChannelCreateRequest) { - handleChannelCreateRequest((GridSelectorNioSessionImpl)ses, connKey, - (ChannelCreateRequest)msg); - - if (c != null) - c.run(); - } - else if (msg instanceof ChannelCreateResponse) { - // msg will be ignored. - handleChannelCreateResponse((GridSelectorNioSessionImpl)ses, connKey); - - if (c != null) - c.run(); - } - else - notifyListener(connKey.nodeId(), msg, c); + notifyListener(connKey.nodeId(), msg, c); } } @@ -1272,9 +1268,6 @@ class ConnectClosure implements IgniteInClosure { /** Clients. */ private final ConcurrentMap clients = GridConcurrentFactory.newMap(); - /** Channel creation local requests (registered on #openChannel()) */ - private final ConcurrentMap> channelReqs = new ConcurrentHashMap<>(); - /** SPI listener. */ private volatile CommunicationListenerEx lsnr; @@ -4383,13 +4376,7 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc "[nodeId=" + remote.id() + ']'; ConnectionKey key = new ConnectionKey(remote.id(), chConnPlc.connectionIndex()); - - if (channelReqs.get(key) != null) { - throw new IgniteSpiException("The channel connection cannot be established to remote node. " + - "Connection key already in use [key=" + key + ']'); - } - - GridFutureAdapter result = new GridFutureAdapter<>(); + GridFutureAdapter chFut = new GridFutureAdapter<>(); connectGate.enter(); @@ -4400,13 +4387,12 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc cleanupLocalNodeRecoveryDescriptor(key); - channelReqs.put(key, result); - + ses.addMeta(CHANNEL_FUT_META, chFut); // Send configuration message over the created session. ses.send(new ChannelCreateRequest(initMsg)) .listen(f -> { if (f.error() != null) { - GridFutureAdapter rq = channelReqs.remove(key); + GridFutureAdapter rq = ses.meta(CHANNEL_FUT_META); assert rq != null; @@ -4428,10 +4414,9 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc @Override public void onTimeout() { // Close session if request not complete yet. - GridFutureAdapter rq = channelReqs.remove(key); + GridFutureAdapter rq = ses.meta(CHANNEL_FUT_META); - if (rq == null) - return; + assert rq != null; if (rq.onDone(handshakeTimeoutException())) ses.close(); @@ -4439,7 +4424,7 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc }); }); - return result; + return chFut; } catch (IgniteCheckedException e) { throw new IgniteSpiException("Unable to create new channel connection to the remote node: " + remote, e); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 77f3302558f48..442b48ecaa8db 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -51,6 +51,7 @@ import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator; @@ -273,7 +274,7 @@ public void testFileHandlerFilePathThrowsEx() throws Exception { /** * @throws Exception If fails. */ - @Test(expected = IgniteException.class) + @Test(expected = IgniteCheckedException.class) public void testFileHandlerOnReceiverLeft() throws Exception { final int fileSizeBytes = 5 * 1024 * 1024; final AtomicInteger chunksCnt = new AtomicInteger(); @@ -428,7 +429,7 @@ public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { err = e; } - assertEquals(IgniteException.class, err.getClass()); + assertEquals(NodeStoppingException.class, err.getClass()); assertEquals("Uncomplete resources must be cleaned up on sender left", 0, fileCount(downloadTo.toPath())); From d2a50bf950b09cb010412c74621006fa1d103924 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Wed, 7 Aug 2019 14:36:27 +0300 Subject: [PATCH 56/69] IGNITE-10619: fix communication listener ex --- .../tcp/internal/channel/CommunicationListenerEx.java | 4 +++- .../spi/communication/GridAbstractCommunicationSelfTest.java | 3 ++- .../tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java | 4 ++-- .../communication/tcp/GridTcpCommunicationSpiLanLoadTest.java | 4 ++-- .../tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java | 4 ++-- .../tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java | 4 ++-- .../tcp/GridTcpCommunicationSpiRecoverySelfTest.java | 4 ++-- .../tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java | 4 ++-- 8 files changed, 17 insertions(+), 14 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java index 46389c8098209..3e188568694a3 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java @@ -31,5 +31,7 @@ public interface CommunicationListenerEx extends Communi * @param initMsg Init channel message. * @param channel Locally created channel endpoint. */ - public void onChannelOpened(UUID nodeId, T initMsg, Channel channel); + public default void onChannelOpened(UUID nodeId, T initMsg, Channel channel) { + // No-op. + } } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java index 8034093c3cc44..cb82636a583b1 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java @@ -37,6 +37,7 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; +import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -89,7 +90,7 @@ public abstract class GridAbstractCommunicationSelfTest { + private class MessageListener implements CommunicationListenerEx { /** */ private final UUID locNodeId; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java index 7ab0d6fb819f8..599ce4e9d89f1 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java @@ -48,9 +48,9 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; -import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; +import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -118,7 +118,7 @@ public GridTcpCommunicationSpiConcurrentConnectSelfTest() { /** * */ - private static class MessageListener implements CommunicationListener { + private static class MessageListener implements CommunicationListenerEx { /** */ private final CountDownLatch latch; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java index b4bc2e9c3dd90..a7925ec55dfb0 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java @@ -33,8 +33,8 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.GridTestMessage; +import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -86,7 +86,7 @@ public GridTcpCommunicationSpiLanLoadTest() { /** * Accumulating listener. */ - private class MessageListener implements CommunicationListener { + private class MessageListener implements CommunicationListenerEx { /** Node id of local node. */ private final UUID locNodeId; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java index cfde86b318f74..1ba21c0c6d1b0 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java @@ -52,9 +52,9 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; -import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; +import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -126,7 +126,7 @@ public GridTcpCommunicationSpiMultithreadedSelfTest() { /** * Accumulating listener. */ - private static class MessageListener implements CommunicationListener { + private static class MessageListener implements CommunicationListenerEx { /** Node id of local node. */ private final UUID locNodeId; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java index 408eb1066d526..00422cff1252e 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java @@ -40,9 +40,9 @@ import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; import org.apache.ignite.spi.IgniteSpiException; -import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; +import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -91,7 +91,7 @@ public GridTcpCommunicationSpiRecoveryAckSelfTest() { } /** */ - private class TestListener implements CommunicationListener { + private class TestListener implements CommunicationListenerEx { /** */ private GridConcurrentHashSet msgIds = new GridConcurrentHashSet<>(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java index 5ec734a017c7c..d7dab63df1793 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java @@ -44,9 +44,9 @@ import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; import org.apache.ignite.spi.IgniteSpiException; -import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; +import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -105,7 +105,7 @@ public GridTcpCommunicationSpiRecoverySelfTest() { } /** */ - private class TestListener implements CommunicationListener { + private class TestListener implements CommunicationListenerEx { /** */ private boolean block; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java index d937bb01ee1cd..94349a5836777 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java @@ -43,9 +43,9 @@ import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; import org.apache.ignite.spi.IgniteSpiException; -import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; +import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -95,7 +95,7 @@ public IgniteTcpCommunicationRecoveryAckClosureSelfTest() { } /** */ - private class TestListener implements CommunicationListener { + private class TestListener implements CommunicationListenerEx { /** */ private GridConcurrentHashSet msgIds = new GridConcurrentHashSet<>(); From 7fdf160e66ff028ea223e8299451fa7e1ded87de Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Wed, 7 Aug 2019 16:24:42 +0300 Subject: [PATCH 57/69] IGNITE-10619: fix communication spi, remove request, response --- .../managers/communication/GridIoManager.java | 2 +- .../communication/GridIoMessageFactory.java | 12 -- .../communication/SessionChannelMessage.java | 2 +- .../tcp/TcpCommunicationSpi.java | 87 ++++++------ .../CommunicationListenerEx.java | 2 +- .../channel/ChannelCreateRequest.java | 134 ------------------ .../channel/ChannelCreateResponse.java | 102 ------------- .../GridAbstractCommunicationSelfTest.java | 2 +- ...unicationSpiConcurrentConnectSelfTest.java | 2 +- .../GridTcpCommunicationSpiLanLoadTest.java | 2 +- ...CommunicationSpiMultithreadedSelfTest.java | 2 +- ...cpCommunicationSpiRecoveryAckSelfTest.java | 2 +- ...idTcpCommunicationSpiRecoverySelfTest.java | 2 +- ...mmunicationRecoveryAckClosureSelfTest.java | 2 +- 14 files changed, 51 insertions(+), 304 deletions(-) rename modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/{channel => }/CommunicationListenerEx.java (95%) delete mode 100644 modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateRequest.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateResponse.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 08fc415713e4c..a04c6ce7b879e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -132,7 +132,7 @@ import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; -import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; +import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java index 5d531e5484091..d8d62d4595a96 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java @@ -200,8 +200,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.collision.jobstealing.JobStealingRequest; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; -import org.apache.ignite.spi.communication.tcp.internal.channel.ChannelCreateRequest; -import org.apache.ignite.spi.communication.tcp.internal.channel.ChannelCreateResponse; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage2; import org.apache.ignite.spi.communication.tcp.messages.HandshakeWaitMessage; @@ -1163,16 +1161,6 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; - case ChannelCreateResponse.TYPE_CODE: - msg = new ChannelCreateResponse(); - - break; - - case ChannelCreateRequest.TYPE_CODE: - msg = new ChannelCreateRequest(); - - break; - case SessionChannelMessage.TYPE_CODE: msg = new SessionChannelMessage(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java index 2fcd73baa3374..4fb92bcdbd216 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java @@ -32,7 +32,7 @@ */ class SessionChannelMessage implements Message { /** Initial channel message type (value is {@code 177}). */ - public static final short TYPE_CODE = 177; + public static final short TYPE_CODE = 175; /** Serialization version. */ private static final long serialVersionUID = 0L; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 642df530b0d0c..8ba40ce15f1f7 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -145,13 +145,11 @@ import org.apache.ignite.spi.TimeoutStrategy; import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; +import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.spi.communication.tcp.internal.ConnectionKey; import org.apache.ignite.spi.communication.tcp.internal.HandshakeException; import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture; import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationNodeConnectionCheckFuture; -import org.apache.ignite.spi.communication.tcp.internal.channel.ChannelCreateRequest; -import org.apache.ignite.spi.communication.tcp.internal.channel.ChannelCreateResponse; -import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage2; import org.apache.ignite.spi.communication.tcp.messages.HandshakeWaitMessage; @@ -359,7 +357,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati /** Message tracker meta for session. */ private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey(); - /** Session future of channel creation request. */ + /** Channel meta used for establishing channel connections. */ private static final int CHANNEL_FUT_META = GridNioSessionMetaKey.nextUniqueKey(); /** @@ -755,33 +753,14 @@ private void closeStaleConnections(ConnectionKey connKey) { } } - private void handleChannelCreateResponse( - GridSelectorNioSessionImpl ses, - GridFutureAdapter reqFut - ) { - assert reqFut != null; - - ses.closeSocketOnSessionClose(false); - - ses.close().listen(f -> { - if (f.error() != null) { - reqFut.onDone(f.error()); - - return; - } - - reqFut.onDone(ses.key().channel()); - }); - } - private void handleChannelCreateRequest( GridSelectorNioSessionImpl ses, ConnectionKey connKey, - ChannelCreateRequest msg + Message msg ) { cleanupLocalNodeRecoveryDescriptor(connKey); - ses.send(new ChannelCreateResponse()) + ses.send(msg) .listen(sendFut -> { if (sendFut.error() != null) { U.error(log, "Fail to send channel creation response to the remote node. " + @@ -807,7 +786,7 @@ private void handleChannelCreateRequest( return; } - notifyChannelEvtListener(connKey.nodeId(), ses.key().channel(), msg.message()); + notifyChannelEvtListener(connKey.nodeId(), ses.key().channel(), msg); }); }); } @@ -839,6 +818,29 @@ private void handleChannelCreateRequest( } } else { + if (isChannelConnIdx(connKey.connectionIndex())) { + if (ses.meta(CHANNEL_FUT_META) == null) + handleChannelCreateRequest((GridSelectorNioSessionImpl)ses, connKey, msg); + else { + GridFutureAdapter fut = ses.meta(CHANNEL_FUT_META); + GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses; + + ses0.closeSocketOnSessionClose(false); + + ses0.close().listen(f -> { + if (f.error() != null) { + fut.onDone(f.error()); + + return; + } + + fut.onDone(ses0.key().channel()); + }); + } + + return; + } + if (msg instanceof RecoveryLastReceivedMessage) { metricsLsnr.onMessageReceived(msg, connKey.nodeId()); @@ -893,22 +895,6 @@ else if (connKey.dummy()) { metricsLsnr.onMessageReceived(msg, connKey.nodeId()); - if (ses.meta(CHANNEL_FUT_META) != null) { - //Response message received and will be ignored - assert msg instanceof ChannelCreateResponse; - - handleChannelCreateResponse((GridSelectorNioSessionImpl)ses, ses.meta(CHANNEL_FUT_META)); - - return; - } - - if (msg instanceof ChannelCreateRequest) { - handleChannelCreateRequest((GridSelectorNioSessionImpl)ses, connKey, - (ChannelCreateRequest)msg); - - return; - } - IgniteRunnable c; if (msgQueueLimit > 0) { @@ -2238,10 +2224,10 @@ private void dumpInfo(StringBuilder sb, UUID dstNodeId) { chConnPlc = new ConnectionPolicy() { /** Sequential connection index provider. */ - private final AtomicInteger connIdx = new AtomicInteger(connectionsPerNode + 1); + private final AtomicInteger chIdx = new AtomicInteger(MAX_CONN_PER_NODE + 1); @Override public int connectionIndex() { - return connIdx.incrementAndGet(); + return chIdx.incrementAndGet(); } }; @@ -4376,6 +4362,7 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc "[nodeId=" + remote.id() + ']'; ConnectionKey key = new ConnectionKey(remote.id(), chConnPlc.connectionIndex()); + GridFutureAdapter chFut = new GridFutureAdapter<>(); connectGate.enter(); @@ -4386,10 +4373,10 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc assert ses != null : "Session must be established [remoteId=" + remote.id() + ", key=" + key + ']'; cleanupLocalNodeRecoveryDescriptor(key); - ses.addMeta(CHANNEL_FUT_META, chFut); + // Send configuration message over the created session. - ses.send(new ChannelCreateRequest(initMsg)) + ses.send(initMsg) .listen(f -> { if (f.error() != null) { GridFutureAdapter rq = ses.meta(CHANNEL_FUT_META); @@ -4434,6 +4421,14 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc } } + /** + * @param connIdx Connection index to check. + * @return {@code true} if connection index is related to the channel create request\response. + */ + private boolean isChannelConnIdx(int connIdx) { + return connIdx > MAX_CONN_PER_NODE; + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/CommunicationListenerEx.java similarity index 95% rename from modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java rename to modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/CommunicationListenerEx.java index 3e188568694a3..d369ff7b92a01 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/CommunicationListenerEx.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/CommunicationListenerEx.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.spi.communication.tcp.internal.channel; +package org.apache.ignite.spi.communication.tcp.internal; import java.io.Serializable; import java.nio.channels.Channel; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateRequest.java deleted file mode 100644 index 2739030ad926e..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateRequest.java +++ /dev/null @@ -1,134 +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.apache.ignite.spi.communication.tcp.internal.channel; - -import java.io.Externalizable; -import java.nio.ByteBuffer; -import java.nio.channels.Channel; -import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageReader; -import org.apache.ignite.plugin.extensions.communication.MessageWriter; - -/** - * Message requesting to creation of {@link Channel}. - */ -public class ChannelCreateRequest implements Message { - /** Request message type (value is {@code 176}). */ - public static final short TYPE_CODE = 176; - - /** Serialization version. */ - private static final long serialVersionUID = 0L; - - /** Initialization channel message which contains channel params. */ - private Message msg; - - /** - * No-op constructor to support {@link Externalizable} interface. - * This constructor is not meant to be used for other purposes. - */ - public ChannelCreateRequest() { - // No-op. - } - - /** - * @param msg Initial channel message, containing channel attributes. - */ - public ChannelCreateRequest(Message msg) { - this.msg = msg; - } - - /** - * @return Channel initialization message. - */ - public Message message() { - return msg; - } - - /** - * @param msg Channel initialization message. - * @return {@code this} for chaining. - */ - public ChannelCreateRequest message(Message msg) { - this.msg = msg; - return this; - } - - /** {@inheritDoc} */ - @Override public void onAckReceived() { - // No-op. - } - - /** {@inheritDoc} */ - @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { - writer.setBuffer(buf); - - if (!writer.isHeaderWritten()) { - if (!writer.writeHeader(directType(), fieldsCount())) - return false; - - writer.onHeaderWritten(); - } - - switch (writer.state()) { - case 0: - if (!writer.writeMessage("msg", msg)) - return false; - - writer.incrementState(); - } - - return true; - } - - /** {@inheritDoc} */ - @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { - reader.setBuffer(buf); - - if (!reader.beforeMessageRead()) - return false; - - switch (reader.state()) { - case 0: - msg = reader.readMessage("msg"); - - if (!reader.isLastRead()) - return false; - - reader.incrementState(); - } - - return reader.afterMessageRead(ChannelCreateRequest.class); - } - - /** {@inheritDoc} */ - @Override public short directType() { - return TYPE_CODE; - } - - /** {@inheritDoc} */ - @Override public byte fieldsCount() { - return 1; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(ChannelCreateRequest.class, this); - } - -} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateResponse.java deleted file mode 100644 index 356c5f0eeccc2..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/channel/ChannelCreateResponse.java +++ /dev/null @@ -1,102 +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.apache.ignite.spi.communication.tcp.internal.channel; - -import java.nio.ByteBuffer; -import java.nio.channels.Channel; -import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageReader; -import org.apache.ignite.plugin.extensions.communication.MessageWriter; - -/** - * Message response to creation of {@link Channel}. - */ -public class ChannelCreateResponse implements Message { - /** Request message type (value is {@code 175}). */ - public static final short TYPE_CODE = 175; - - /** Serialization version. */ - private static final long serialVersionUID = 0L; - - /** Ack processing byte (used as message content). */ - private byte ack; - - /** {@inheritDoc} */ - @Override public void onAckReceived() { - // No-op. - } - - /** {@inheritDoc} */ - @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { - writer.setBuffer(buf); - - if (!writer.isHeaderWritten()) { - if (!writer.writeHeader(directType(), fieldsCount())) - return false; - - writer.onHeaderWritten(); - } - - switch (writer.state()) { - case 0: - if (!writer.writeByte("ack", ack)) - return false; - - writer.incrementState(); - } - - return true; - } - - /** {@inheritDoc} */ - @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { - reader.setBuffer(buf); - - if (!reader.beforeMessageRead()) - return false; - - switch (reader.state()) { - case 0: - ack = reader.readByte("ack"); - - if (!reader.isLastRead()) - return false; - - reader.incrementState(); - } - - return reader.afterMessageRead(ChannelCreateResponse.class); - } - - /** {@inheritDoc} */ - @Override public short directType() { - return TYPE_CODE; - } - - /** {@inheritDoc} */ - @Override public byte fieldsCount() { - return 1; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(ChannelCreateResponse.class, this); - } - -} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java index cb82636a583b1..d468489198ccf 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java @@ -37,7 +37,7 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; -import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; +import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java index 599ce4e9d89f1..4fbda4e99e02d 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java @@ -50,7 +50,7 @@ import org.apache.ignite.spi.IgniteSpiAdapter; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; +import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java index a7925ec55dfb0..bd50c5ba167ff 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java @@ -34,7 +34,7 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; +import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java index 1ba21c0c6d1b0..fc756ff25ba7f 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java @@ -54,7 +54,7 @@ import org.apache.ignite.spi.IgniteSpiAdapter; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; +import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java index 00422cff1252e..c879114b9b385 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java @@ -42,7 +42,7 @@ import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; +import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java index d7dab63df1793..f306b1e5ce74d 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java @@ -46,7 +46,7 @@ import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; +import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java index 94349a5836777..b145b4a091402 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java @@ -45,7 +45,7 @@ import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.channel.CommunicationListenerEx; +import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; From 72a01316f4840e150ac56d78e0a81efca848594d Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Wed, 7 Aug 2019 16:48:22 +0300 Subject: [PATCH 58/69] IGNITE-10619: rename channel create method --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 8ba40ce15f1f7..e68376d1a41a8 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -753,7 +753,7 @@ private void closeStaleConnections(ConnectionKey connKey) { } } - private void handleChannelCreateRequest( + private void onChannelCreate( GridSelectorNioSessionImpl ses, ConnectionKey connKey, Message msg @@ -820,7 +820,7 @@ private void handleChannelCreateRequest( else { if (isChannelConnIdx(connKey.connectionIndex())) { if (ses.meta(CHANNEL_FUT_META) == null) - handleChannelCreateRequest((GridSelectorNioSessionImpl)ses, connKey, msg); + onChannelCreate((GridSelectorNioSessionImpl)ses, connKey, msg); else { GridFutureAdapter fut = ses.meta(CHANNEL_FUT_META); GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses; From b4661c6cd9f484e8b129aaf87762eb9ce6d1b792 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Wed, 7 Aug 2019 16:56:25 +0300 Subject: [PATCH 59/69] IGNITE-10619: minor code changes --- .../managers/communication/GridIoManager.java | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index a04c6ce7b879e..ecdcf2799c96a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -3274,17 +3274,6 @@ public void send( ", transferred=" + snd.transferred() + ", rmtId=" + rmtId +']'); } - catch (IgniteException e) { - closeChannelQuiet(); - - if (stopping) - throw new NodeStoppingException("Operation has been cancelled (node is stopping)"); - - if (senderStopFlags.get(sesKey).get()) - throw new ClusterTopologyCheckedException("Remote node left the cluster: " + rmtId, e); - - throw e; - } catch (InterruptedException e) { closeChannelQuiet(); @@ -3299,6 +3288,12 @@ public void send( catch (Throwable e) { closeChannelQuiet(); + if (stopping) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)"); + + if (senderStopFlags.get(sesKey).get()) + throw new ClusterTopologyCheckedException("Remote node left the cluster: " + rmtId, e); + throw new IgniteException("Unexpected exception while sending file to the remote node. The process stopped " + "[rmtId=" + rmtId + ", file=" + file.getName() + ", sesKey=" + sesKey + ']', e); } From 6676f258aad63720e85dd46594d970fa7c1622a5 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 8 Aug 2019 12:03:24 +0300 Subject: [PATCH 60/69] IGNITE-10619: minor code changes after review --- .../managers/communication/ChunkReceiver.java | 18 +++++++++--------- .../managers/communication/FileReceiver.java | 13 +++++++------ .../managers/communication/GridIoManager.java | 4 ++-- .../GridIoManagerFileTransmissionSelfTest.java | 8 ++++---- 4 files changed, 22 insertions(+), 21 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java index 967aee838f32e..6ae58f2d7c6f1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java @@ -64,38 +64,38 @@ public ChunkReceiver( /** {@inheritDoc} */ @Override protected void readChunk(ReadableByteChannel ch) throws IOException { - assert buf != null : "Buffer is used to deilver readed data to the used and cannot be null: " + this; + assert buf != null : "Buffer cannot be null since it is used to receive the data from channel: " + this; buf.rewind(); - int readed = 0; + int read = 0; int res; - // Read data from input channel utill the buffer will be completely filled + // Read data from input channel until the buffer will be completely filled // (buf.remaining() returns 0) or partitially filled buffer if it was the last chunk. while (true) { res = ch.read(buf); // Read will return -1 if remote node close connection. if (res < 0) { - if (transferred + readed != meta.count()) { + if (transferred + read != meta.count()) { throw new IOException("Input data channel reached its end, but file has not fully loaded " + - "[transferred=" + transferred + ", readed=" + readed + ", total=" + meta.count() + ']'); + "[transferred=" + transferred + ", read=" + read + ", total=" + meta.count() + ']'); } break; } - readed += res; + read += res; - if (readed == buf.capacity() || buf.position() == buf.capacity()) + if (read == buf.capacity() || buf.position() == buf.capacity()) break; } - if (readed == 0) + if (read == 0) return; - transferred += readed; + transferred += read; buf.flip(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index b2de357aba048..996971a845fe7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -34,8 +34,9 @@ import org.apache.ignite.internal.util.typedef.internal.U; /** - * Class represents a chunk data receiver which is pulling data from channel vi - * {@link FileChannel#transferFrom(ReadableByteChannel, long, long)}. + * Class represents the data receiver which is pulling data from channel using + * {@link FileChannel#transferFrom(ReadableByteChannel, long, long)} until the + * whole file will be completely received. */ class FileReceiver extends TransmissionReceiver { /** Handler to notify when a file has been processed. */ @@ -99,13 +100,13 @@ public FileReceiver( long batchSize = Math.min(chunkSize, meta.count() - transferred); - long readed = fileIo.transferFrom(ch, meta.offset() + transferred, batchSize); + long read = fileIo.transferFrom(ch, meta.offset() + transferred, batchSize); - if (readed == 0) + if (read == 0) throw new IOException("Channel is reached the end of stream. Probably, channel is closed on the remote node"); - if (readed > 0) - transferred += readed; + if (read > 0) + transferred += read; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index ecdcf2799c96a..9be351789b69d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -2753,7 +2753,7 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(nodeId, hnd, newSesId)); } - // Do not allow multiple connection for the same session + // Do not allow multiple connection for the same session. if (!newSesId.equals(rcvCtx.sesId)) { IgniteCheckedException err = new IgniteCheckedException("Requested topic is busy by another transmission. " + "It's not allowed to process different sessions over the same topic simultaneously. " + @@ -2793,7 +2793,7 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa catch (Throwable t) { U.error(log, "Download session cannot be finished due to an unexpected error [ctx=" + rcvCtx + ']', t); - // Do not remove receiver context here, since sender will recconect to get this error + // Do not remove receiver context here, since sender will recconect to get this error. interruptRecevier(rcvCtx, new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t)); } finally { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 442b48ecaa8db..9cff602bd83eb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -448,7 +448,7 @@ public void testFileHandlerReconnectOnReadFail() throws Exception { snd.cluster().active(true); File fileToSend = createFileRandomData("testFile", 5 * 1024 * 1024); - final AtomicInteger readedChunks = new AtomicInteger(); + final AtomicInteger readChunks = new AtomicInteger(); transmissionFileIoFactory(rcv, new FileIOFactory() { @Override public FileIO create(File file, OpenOption... modes) throws IOException { @@ -459,7 +459,7 @@ public void testFileHandlerReconnectOnReadFail() throws Exception { @Override public long transferFrom(ReadableByteChannel src, long position, long count) throws IOException { // Read 4 chunks than throw an exception to emulate error processing. - if (readedChunks.incrementAndGet() == 4) + if (readChunks.incrementAndGet() == 4) throw new IgniteException(chunkDownloadExMsg); return super.transferFrom(src, position, count); @@ -559,7 +559,7 @@ public void testFileHandlerNextWriterOpened() throws Exception { assertNotNull("Transmission must ends with an exception", expectedErr); - //Open next session and complete successfull. + // Open next session and complete successfull. try (GridIoManager.TransmissionSender sender = snd.context() .io() .openTransmissionSender(rcv.localNode().id(), topic)) { @@ -580,7 +580,7 @@ public void testFileHandlerSendToNullTopic() throws Exception { snd.cluster().active(true); - // Ensure topic handler is empty + // Ensure topic handler is empty. rcv.context().io().removeTransmissionHandler(topic); // Open next writer on removed topic. From d7552a9aeddfdf9a22f7c38592260b6eaf24a0ba Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Thu, 8 Aug 2019 17:54:54 +0300 Subject: [PATCH 61/69] IGNITE-10619: fix javadoc issues --- .../managers/communication/GridIoManager.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 9be351789b69d..40a82f8fcd233 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -1141,10 +1141,11 @@ private void format(StringBuilder b, Collection> pairs, SimpleD } /** - * @param nodeId The remote node id. + * @param rmtNodeId The remote node id. + * @param initMsg Message with additional channel params. * @param channel The channel to notify listeners with. */ - private void onChannelOpened0(UUID nodeId, GridIoMessage initMsg, Channel channel) { + private void onChannelOpened0(UUID rmtNodeId, GridIoMessage initMsg, Channel channel) { Lock busyLock0 = busyLock.readLock(); busyLock0.lock(); @@ -1153,7 +1154,7 @@ private void onChannelOpened0(UUID nodeId, GridIoMessage initMsg, Channel channe if (stopping) { if (log.isDebugEnabled()) { log.debug("Received communication channel create event while node stopping (will ignore) " + - "[nodeId=" + nodeId + ", msg=" + initMsg + ']'); + "[rmtNodeId=" + rmtNodeId + ", initMsg=" + initMsg + ']'); } return; @@ -1170,14 +1171,14 @@ private void onChannelOpened0(UUID nodeId, GridIoMessage initMsg, Channel channe pools.poolForPolicy(plc).execute(new Runnable() { @Override public void run() { - processOpenedChannel(initMsg.topic(), nodeId, (SessionChannelMessage)initMsg.message(), + processOpenedChannel(initMsg.topic(), rmtNodeId, (SessionChannelMessage)initMsg.message(), (SocketChannel)channel); } }); } catch (IgniteCheckedException e) { U.error(log, "Failed to process channel creation event due to exception " + - "[nodeId=" + nodeId + ", initMsg=" + initMsg + ']' , e); + "[rmtNodeId=" + rmtNodeId + ", initMsg=" + initMsg + ']' , e); } finally { busyLock0.unlock(); From e0ef04745cc649097b562bacb7ad61d6ffe8e358 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 9 Aug 2019 12:04:59 +0300 Subject: [PATCH 62/69] IGNITE-10619: rename nodeId to rmtNodeId --- .../managers/communication/GridIoManager.java | 38 +++++++++---------- .../tcp/internal/CommunicationListenerEx.java | 4 +- 2 files changed, 21 insertions(+), 21 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 40a82f8fcd233..ef00497afa552 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -448,9 +448,9 @@ public void resetMetrics() { lsnr.onNodeDisconnected(nodeId); } - @Override public void onChannelOpened(UUID nodeId, Serializable initMsg, Channel channel) { + @Override public void onChannelOpened(UUID rmtNodeId, Serializable initMsg, Channel channel) { try { - onChannelOpened0(nodeId, (GridIoMessage)initMsg, channel); + onChannelOpened0(rmtNodeId, (GridIoMessage)initMsg, channel); } catch (ClassCastException ignored) { U.error(log, "Communication manager received message of unknown type (will ignore): " + @@ -937,7 +937,7 @@ private void format(StringBuilder b, Collection> pairs, SimpleD while (it.hasNext()) { Map.Entry e = it.next(); - if (nodeId.equals(e.getValue().nodeId)) { + if (nodeId.equals(e.getValue().rmtNodeId)) { it.remove(); interruptRecevier(e.getValue(), @@ -2704,7 +2704,7 @@ private void interruptRecevier(ReceiverContext rctx, Exception ex) { rctx.lastState = rctx.lastState == null ? new TransmissionMeta(ex) : rctx.lastState.error(ex); - rctx.hnd.onException(rctx.nodeId, ex); + rctx.hnd.onException(rctx.rmtNodeId, ex); U.error(log, "Receiver has been interrupted due to an excpetion occurred [ctx=" + rctx + ']', ex); } @@ -2712,11 +2712,11 @@ private void interruptRecevier(ReceiverContext rctx, Exception ex) { /** * @param topic Topic to which the channel is created. - * @param nodeId Remote node id. + * @param rmtNodeId Remote node id. * @param initMsg Channel initialization message with additional params. * @param ch Channel instance. */ - private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessage initMsg, SocketChannel ch) { + private void processOpenedChannel(Object topic, UUID rmtNodeId, SessionChannelMessage initMsg, SocketChannel ch) { ReceiverContext rcvCtx = null; ObjectInputStream in = null; ObjectOutputStream out = null; @@ -2729,7 +2729,7 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa if (initMsg == null || initMsg.sesId() == null) { U.warn(log, "There is no initial message provied for given topic. Opened channel will be closed " + - "[nodeId=" + nodeId + ", topic=" + topic + ", initMsg=" + initMsg + ']'); + "[rmtNodeId=" + rmtNodeId + ", topic=" + topic + ", initMsg=" + initMsg + ']'); return; } @@ -2745,20 +2745,20 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa TransmissionHandler hnd = topicTransmissionHnds.get(topic); if (hnd == null) { - U.warn(log, "There is no handler for a given topic. Channel will be closed [nodeId=" + nodeId + + U.warn(log, "There is no handler for a given topic. Channel will be closed [rmtNodeId=" + rmtNodeId + ", topic=" + topic + ']'); return; } - rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(nodeId, hnd, newSesId)); + rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(rmtNodeId, hnd, newSesId)); } // Do not allow multiple connection for the same session. if (!newSesId.equals(rcvCtx.sesId)) { IgniteCheckedException err = new IgniteCheckedException("Requested topic is busy by another transmission. " + "It's not allowed to process different sessions over the same topic simultaneously. " + - "Channel will be closed [initMsg=" + initMsg + ", channel=" + ch + ", nodeId=" + nodeId + ']'); + "Channel will be closed [initMsg=" + initMsg + ", channel=" + ch + ", nodeId=" + rmtNodeId + ']'); U.error(log, err); @@ -2768,7 +2768,7 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa } if (log.isDebugEnabled()) { - log.debug("Trasmission open a new channel [nodeId=" + nodeId + ", topic=" + topic + + log.debug("Trasmission open a new channel [rmtNodeId=" + rmtNodeId + ", topic=" + topic + ", initMsg=" + initMsg + ']'); } @@ -2795,7 +2795,7 @@ private void processOpenedChannel(Object topic, UUID nodeId, SessionChannelMessa U.error(log, "Download session cannot be finished due to an unexpected error [ctx=" + rcvCtx + ']', t); // Do not remove receiver context here, since sender will recconect to get this error. - interruptRecevier(rcvCtx, new IgniteCheckedException("Channel processing error [nodeId=" + nodeId + ']', t)); + interruptRecevier(rcvCtx, new IgniteCheckedException("Channel processing error [nodeId=" + rmtNodeId + ']', t)); } finally { U.closeQuiet(in); @@ -2836,7 +2836,7 @@ private void receiveFromChannel( TransmissionMeta meta = (TransmissionMeta)in.readObject(); if (rcvCtx.rcv == null) { - rcvCtx.rcv = createReceiver(rcvCtx.nodeId, + rcvCtx.rcv = createReceiver(rcvCtx.rmtNodeId, rcvCtx.hnd, meta, () -> stopping || rcvCtx.interrupted.get()); @@ -2860,7 +2860,7 @@ private void receiveFromChannel( U.log(log, "File has been received " + "[name=" + rcvCtx.rcv.state().name() + ", transferred=" + rcvCtx.rcv.transferred() + ", time=" + (double)((U.currentTimeMillis() - startTime) / 1000) + " sec" + - ", rmtId=" + rcvCtx.nodeId + ']'); + ", rmtId=" + rcvCtx.rmtNodeId + ']'); rcvCtx.rcv = null; } @@ -2877,7 +2877,7 @@ private void receiveFromChannel( catch (IOException e) { // Waiting for re-establishing connection. U.warn(log, "Сonnection from the remote node lost. Will wait for the new one to continue file " + - "receive [nodeId=" + rcvCtx.nodeId + ", sesKey=" + rcvCtx.sesId + ']', e); + "receive [nodeId=" + rcvCtx.rmtNodeId + ", sesKey=" + rcvCtx.sesId + ']', e); long startTs = U.currentTimeMillis(); @@ -2994,7 +2994,7 @@ public void dumpStats() { */ private static class ReceiverContext { /** The remote node input channel came from. */ - private final UUID nodeId; + private final UUID rmtNodeId; /** Current sesssion handler. */ @GridToStringExclude @@ -3019,12 +3019,12 @@ private static class ReceiverContext { private GridTimeoutObject timeoutObj; /** - * @param nodeId Remote node id. + * @param rmtNodeId Remote node id. * @param hnd Channel handler of current topic. * @param sesId Unique session request id. */ - public ReceiverContext(UUID nodeId, TransmissionHandler hnd, IgniteUuid sesId) { - this.nodeId = nodeId; + public ReceiverContext(UUID rmtNodeId, TransmissionHandler hnd, IgniteUuid sesId) { + this.rmtNodeId = rmtNodeId; this.hnd = hnd; this.sesId = sesId; } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/CommunicationListenerEx.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/CommunicationListenerEx.java index d369ff7b92a01..f6022c21e3254 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/CommunicationListenerEx.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/CommunicationListenerEx.java @@ -27,11 +27,11 @@ */ public interface CommunicationListenerEx extends CommunicationListener { /** - * @param nodeId Remote node id. + * @param rmtNodeId Remote node id. * @param initMsg Init channel message. * @param channel Locally created channel endpoint. */ - public default void onChannelOpened(UUID nodeId, T initMsg, Channel channel) { + public default void onChannelOpened(UUID rmtNodeId, T initMsg, Channel channel) { // No-op. } } From 4878ca8c4228e9183e12b26b627f7a88371f6a63 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 9 Aug 2019 12:46:54 +0300 Subject: [PATCH 63/69] IGNITE-10619: add resources checks for tests --- ...GridIoManagerFileTransmissionSelfTest.java | 88 +++++++++++++------ 1 file changed, 62 insertions(+), 26 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 9cff602bd83eb..7db6028ea6655 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -31,6 +31,7 @@ import java.util.HashMap; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; @@ -59,7 +60,9 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; @@ -102,6 +105,12 @@ public class GridIoManagerFileTransmissionSelfTest extends GridCommonAbstractTes /** The temporary directory to store files. */ private File tempStore; + /** Ignite instance which receives files. */ + private IgniteEx rcv; + + /** Ignite instance which sends files. */ + private IgniteEx snd; + /** * Called before tests started. */ @@ -131,8 +140,18 @@ public void before() throws Exception { */ @After public void after() { - stopAllGrids(); - U.closeQuiet(fileIo[0]); + try { + ensureResourcesFree(snd); + ensureResourcesFree(rcv); + } + finally { + stopAllGrids(); + + U.closeQuiet(fileIo[0]); + + snd = null; + rcv = null; + } } /** {@inheritDoc} */ @@ -154,8 +173,8 @@ public void after() { */ @Test public void testFileHandlerBase() throws Exception { - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -235,8 +254,8 @@ public void testFileHandlerBase() throws Exception { public void testFileHandlerFilePathThrowsEx() throws Exception { final String exTestMessage = "Test exception. Handler initialization failed at onBegin."; - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -279,8 +298,8 @@ public void testFileHandlerOnReceiverLeft() throws Exception { final int fileSizeBytes = 5 * 1024 * 1024; final AtomicInteger chunksCnt = new AtomicInteger(); - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -319,8 +338,8 @@ public void testFileHandlerOnReceiverLeft() throws Exception { */ @Test public void tesFileHandlerReconnectTimeouted() throws Exception { - IgniteEx rcv = startGrid(1); - IgniteEx snd = startGrid(0); + rcv = startGrid(1); + snd = startGrid(0); final AtomicInteger chunksCnt = new AtomicInteger(); final CountDownLatch sndLatch = ((BlockingOpenChannelCommunicationSpi)snd.context() @@ -383,8 +402,8 @@ public void tesFileHandlerReconnectTimeouted() throws Exception { */ @Test public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -442,8 +461,8 @@ public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { public void testFileHandlerReconnectOnReadFail() throws Exception { final String chunkDownloadExMsg = "Test exception. Chunk processing error."; - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -489,8 +508,8 @@ public void testFileHandlerSenderStoppedIfReceiverInitFail() throws Exception { final int fileSizeBytes = 5 * 1024 * 1024; final AtomicBoolean throwFirstTime = new AtomicBoolean(); - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -524,8 +543,8 @@ public void testFileHandlerNextWriterOpened() throws Exception { final int fileSizeBytes = 5 * 1024 * 1024; final AtomicBoolean networkExThrown = new AtomicBoolean(); - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -575,8 +594,8 @@ public void testFileHandlerNextWriterOpened() throws Exception { */ @Test(expected = IgniteException.class) public void testFileHandlerSendToNullTopic() throws Exception { - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -600,8 +619,8 @@ public void testFileHandlerChannelCloseIfAnotherOpened() throws Exception { final CountDownLatch waitLatch = new CountDownLatch(2); final CountDownLatch completionWait = new CountDownLatch(2); - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -620,6 +639,7 @@ public void testFileHandlerChannelCloseIfAnotherOpened() throws Exception { try (GridIoManager.TransmissionSender sender = snd.context() .io() .openTransmissionSender(rcv.localNode().id(), topic); + GridIoManager.TransmissionSender anotherSender = snd.context() .io() .openTransmissionSender(rcv.localNode().id(), topic)) { @@ -665,8 +685,8 @@ public void testFileHandlerChannelCloseIfAnotherOpened() throws Exception { */ @Test public void testChunkHandlerWithReconnect() throws Exception { - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); final String filePrefix = "testFile"; final AtomicInteger cnt = new AtomicInteger(); @@ -768,8 +788,8 @@ private void closeIfTransferred() { */ @Test(expected = IgniteCheckedException.class) public void testChunkHandlerInitSizeFail() throws Exception { - IgniteEx snd = startGrid(0); - IgniteEx rcv = startGrid(1); + snd = startGrid(0); + rcv = startGrid(1); snd.cluster().active(true); @@ -789,6 +809,22 @@ public void testChunkHandlerInitSizeFail() throws Exception { } } + /** + * @param ig Ignite instance to check. + */ + private static void ensureResourcesFree(IgniteEx ig) { + if (ig == null) + return; + + final GridIoManager io = ig.context().io(); + + ConcurrentMap ctxs = GridTestUtils.getFieldValue(io, "rcvCtxs"); + ConcurrentMap, AtomicBoolean> sndrFlags = GridTestUtils.getFieldValue(io, "senderStopFlags"); + + assertTrue("Receiver context map must be empty: " + ctxs, ctxs.isEmpty()); + assertTrue("Sender stop falgs must be empty: " + sndrFlags, sndrFlags.isEmpty()); + } + /** * @param ignite Ignite instance. * @param cacheName Cache name to add data to. From bce6708f092df24d21414b3b832f384b2e22a36a Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 12 Aug 2019 14:07:49 +0300 Subject: [PATCH 64/69] IGNITE-10619: update minor javadoc issues --- .../managers/communication/GridIoManager.java | 14 +++++++++----- .../communication/TransmissionHandler.java | 3 ++- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index ef00497afa552..e90d95eeee107 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -204,14 +204,15 @@ *

* It is possible to receive a set of files on a particular topic (any of {@link GridTopic}) on the remote node. * A transmission handler for desired topic must be registered prior to opening transmission sender to it. - * Use methods below are used to register handlers and open new transmissions: + * Methods below are used to register handlers and open new transmissions: *

    *
  • {@link #addTransmissionHandler(Object, TransmissionHandler)}
  • + *
  • {@link #removeTransmissionHandler(Object)}
  • *
  • {@link #openTransmissionSender(UUID, Object)}
  • *
*

* Each transmission sender opens a new transmission session to remote node prior to sending files over it. - * (see description of {@link TransmissionSender TransmissionSender} for details). The TransmissionSender + * (see description of {@link TransmissionSender} for details). The TransmissionSender * will send all files within single session syncronously one by one. *

* NOTE. It is important to call close() method or use try-with-resource @@ -3061,7 +3062,7 @@ public ReceiverContext(UUID rmtNodeId, TransmissionHandler hnd, IgniteUuid sesId * *

Exceptions handling

*

- * The transmission can have two different levels of exception which are handled differently: + * Each transmission can have two different high-level types of exception which are handled differently: *

    *
  • transport exception(e.g. some network issues)
  • *
  • application\handler level exception
  • @@ -3069,7 +3070,7 @@ public ReceiverContext(UUID rmtNodeId, TransmissionHandler hnd, IgniteUuid sesId * *

    Application exceptions

    *

    - * The transmission will be stopped immediately and wrapping IgniteCheckedExcpetion thrown in case of + * The transmission will be stopped immediately and wrapping IgniteExcpetion thrown in case of * any application exception occured. * *

    Transport exceptions

    @@ -3083,13 +3084,16 @@ public ReceiverContext(UUID rmtNodeId, TransmissionHandler hnd, IgniteUuid sesId * Another example, the transmission sender gets the Connection reset by peer IOException message. * This means that the remote node you are connected to has to reset the connection. This is usually caused by a * high amount of traffic on the host, but may be caused by a server error or the remote node has exhausted - * system resources as well. Such IOException will be considered as reconnect required. + * system resources as well. Such IOException will be considered as reconnection required. * *

    Timeout exceptions

    *

    * For read operations over the {@link InputStream} or write operation through the {@link OutputStream} * the {@link Socket#setSoTimeout(int)} will be used and an {@link SocketTimeoutException} will be * thrown when the timeout occured. The default value is taken from {@link IgniteConfiguration#getNetworkTimeout()}. + *

    + * If reconnection is not occured withing configured timeout interval the timeout object will be fired which + * clears corresponding to the used topic the {@link ReceiverContext}. * *

    Release resources

    *

    diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java index 8749f18f53821..591a54ce6a72f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java @@ -55,6 +55,7 @@ public interface TransmissionHandler { /** * Chunk handler represents by itself the way of input data stream processing. * It accepts within each chunk a {@link ByteBuffer} with data from input for further processing. + * Activated when the {@link TransmissionPolicy#CHUNK} policy sent. * * @param nodeId Remote node id from which request has been received. * @param initMeta Initial handler meta info. @@ -65,7 +66,7 @@ public interface TransmissionHandler { /** * File handler represents by itself the way of input data stream processing. All the data will * be processed under the hood using zero-copy transferring algorithm and only start file processing and - * the end of processing will be provided. + * the end of processing will be provided. Activated when the {@link TransmissionPolicy#FILE} policy sent. * * @param nodeId Remote node id from which request has been received. * @param initMeta Initial handler meta info. From f91b4237ebb60462f17e0642863c777b5e9ff3be Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 12 Aug 2019 17:19:59 +0300 Subject: [PATCH 65/69] IGNITE-10619: update major javadoc --- .../ignite/internal/IgniteFeatures.java | 2 +- .../managers/communication/ChunkReceiver.java | 15 ++-- .../managers/communication/FileReceiver.java | 4 +- .../managers/communication/FileSender.java | 17 ++-- .../managers/communication/GridIoManager.java | 84 +++++++++---------- .../communication/SessionChannelMessage.java | 4 +- .../communication/TransmissionHandler.java | 10 +-- .../communication/TransmissionMeta.java | 17 ++-- .../communication/TransmissionPolicy.java | 6 +- ...GridIoManagerFileTransmissionSelfTest.java | 36 ++++---- 10 files changed, 93 insertions(+), 102 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java index 59f6dbd033831..7d99cd3297a25 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java @@ -57,7 +57,7 @@ public enum IgniteFeatures { /** Command which allow to detect and cleanup garbage which could left after destroying caches in shared groups */ FIND_AND_DELETE_GARBAGE_COMMAND(8), - /** The node can communicate with other via socket channel. */ + /** The node can communicate with others via socket channel. */ CHANNEL_COMMUNICATION(9), /** Distributed metastorage. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java index 6ae58f2d7c6f1..24aeef2fdbd93 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/ChunkReceiver.java @@ -21,6 +21,7 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SocketChannel; import java.util.function.BooleanSupplier; import java.util.function.Consumer; import org.apache.ignite.IgniteLogger; @@ -28,22 +29,22 @@ import org.apache.ignite.internal.util.typedef.internal.S; /** - * Buffered chunked receiver can handle input socket channel by chunks of data and - * deliver it to an allocated {@link ByteBuffer}. + * Chunk receiver used to read a given {@link SocketChannel} by chunks of predefined size into + * an allocated {@link ByteBuffer}. */ class ChunkReceiver extends TransmissionReceiver { - /** Chunked channel handler to process data with chunks. */ + /** Handler which accepts received data from the given socket. */ private final Consumer hnd; - /** The destination object to transfer data to\from. */ + /** Destination buffer to transfer data to\from. */ private ByteBuffer buf; /** * @param meta Initial file meta info. - * @param chunkSize Size of chunks. + * @param chunkSize Size of the chunk. * @param stopChecker Node stop or prcoess interrupt checker. - * @param hnd Transmission handler to process download result. - * @param log Ignite looger. + * @param hnd Transmission handler to process received data. + * @param log Ignite logger. */ public ChunkReceiver( TransmissionMeta meta, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index 996971a845fe7..6af3ca4286f1e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -39,10 +39,10 @@ * whole file will be completely received. */ class FileReceiver extends TransmissionReceiver { - /** Handler to notify when a file has been processed. */ + /** Handler to notify when a file has been received. */ private final Consumer hnd; - /** The abstract java representation of the chunked file. */ + /** File to receive data into. */ private File file; /** The corresponding file channel to work with. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java index 5e9548d257489..a4c060ff5a712 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileSender.java @@ -38,26 +38,26 @@ /** * Class represents a data sender by chunks of predefined size. All of the chunks will be written to the - * given socket channel. It is important that for each file you are going to send a new FileSender + * given socket channel. Please note, that for each file you are going to send a new {@link FileSender} * instance will be created. The sender must keep its internal state of how much data already being * transferred to send its state to remote node when reconnection required. *

    - * The FileSender uses the zero-copy streaming algorithm, see FileChannel#transferTo for details. + * The FileSender uses the zero-copy streaming approach, see FileChannel#transferTo for details. * * @see FileChannel#transferTo(long, long, WritableByteChannel) */ class FileSender extends AbstractTransmission { - /** Corresponding file channel to work with given file. */ + /** Corresponding file channel to work with a given file. */ @GridToStringExclude private FileIO fileIo; /** - * @param file File which is going to be send by chunks. + * @param file File which is going to be sent by chunks. * @param off File offset. * @param cnt Number of bytes to transfer. * @param params Additional file params. * @param plc Policy of handling data on remote. - * @param stopChecker Node stop or prcoess interrupt checker. + * @param stopChecker Node stop or process interrupt checker. * @param log Ignite logger. * @param factory Factory to produce IO interface on given file. * @param chunkSize Size of chunks. @@ -78,9 +78,7 @@ public FileSender( assert file != null; - // Can be not null if reconnection is going to be occurred. - if (fileIo == null) - fileIo = factory.create(file); + fileIo = factory.create(file); } /** @@ -94,7 +92,6 @@ public void send(WritableByteChannel ch, ObjectOutput oo, @Nullable TransmissionMeta rcvMeta ) throws IOException, InterruptedException { - // If not the initial connection for the current session. updateSenderState(rcvMeta); // Write flag to remote to keep currnet transmission opened. @@ -147,7 +144,7 @@ private void updateSenderState(TransmissionMeta rcvMeta) { transferred = uploadedBytes; - U.log(log, "Update senders number of transferred bytes after reconnect: " + uploadedBytes); + U.log(log, "The number of transferred bytes after reconnect has been updated: " + uploadedBytes); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index e90d95eeee107..c952d686f0f03 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -85,6 +85,7 @@ import org.apache.ignite.internal.IgniteDeploymentCheckedException; import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.direct.DirectMessageReader; @@ -163,23 +164,24 @@ /** * This class represents the internal grid communication (input and output) manager - * which is placed as a layer of indirection interaction between the Ignies Kernal and Communicaiton SPI. - * Communication manager resopnsible for controlling Communication SPI which in turn is responsible - * for exchanging data between Ignites nodes. + * which is placed as a layer of indirection between the {@link IgniteKernal} and {@link CommunicationSpi}. + * The IO manager is responsible for controlling CommunicationSPI which in turn is responsible + * for exchanging data between Ignite nodes. * *

    Data exchanging

    *

    * Communication manager provides a rich API for data exchanging between a pair of cluster nodes. Two types * of communication Message-based communication and File-based communication are available. - * Each of them support sending data to an arbitrary topic on the remote node (topics of {@link GridTopic} is used). + * Each of them support sending data to an arbitrary topic on the remote node (see {@link GridTopic} for an + * additional information about Ignite topics). * *

    Message-based communication

    *

    - * Ignites extension {@link Message} and {@link GridTopic} are used to provide a topic-based messaging protocol + * {@link Message} and {@link GridTopic} are used to provide a topic-based messaging protocol * between cluster nodes. All of messages used for data exchanging can be devided into two general types: * internal and user messages. *

    - * Internal message communication is used by Ignites Kernal. Please, refer to appropriate methods below: + * Internal message communication is used by Ignite Kernal. Please, refer to appropriate methods below: *

      *
    • {@link #sendToGridTopic(ClusterNode, GridTopic, Message, byte)}
    • *
    • {@link #sendOrderedMessage(ClusterNode, Object, Message, byte, long, boolean)}
    • @@ -197,7 +199,7 @@ *

      File-based communication

      *

      * Sending or receiving binary data (represented by a File) over a SocketChannel is only - * possible when the build-in TcpCommunicationSpi implementation of Communication SPI is used and + * possible when the build-in {@link TcpCommunicationSpi} implementation of Communication SPI is used and * both local and remote nodes are {@link IgniteFeatures#CHANNEL_COMMUNICATION CHANNEL_COMMUNICATION} feature * support. To ensue that the remote node satisfies all conditions the {@link #fileTransmissionSupported(ClusterNode)} * method must be called prior to data sending. @@ -212,7 +214,7 @@ *

    *

    * Each transmission sender opens a new transmission session to remote node prior to sending files over it. - * (see description of {@link TransmissionSender} for details). The TransmissionSender + * (see description of {@link TransmissionSender} for details). The {@link TransmissionSender} * will send all files within single session syncronously one by one. *

    * NOTE. It is important to call close() method or use try-with-resource @@ -259,7 +261,7 @@ public class GridIoManager extends GridManagerAdapter CUR_PLC = new ThreadLocal<>(); /** - * Default transfer chunk size in bytes used for sending\receiving files over a SocketChannel. + * Default chunk size in bytes used for sending\receiving files over a {@link SocketChannel}. * Setting the transfer chunk size more than 1 MB is meaningless because there is * no asymptotic benefit. What you're trying to achieve with larger transfer chunk sizes is * fewer thread context switches, and every time we double the transfer size you have @@ -269,7 +271,7 @@ public class GridIoManager extends GridManagerAdapter, AtomicBoolean> senderStopFlags = new ConcurrentHashMap<>(); /** - * Default factory to provide IO operation interface over files for futher transmission them between nodes. - * Some implementations of file senders\receivers are using the zero-copy algorithm to tranasfer bytes - * from a file to the given SocketChannel and vice-versa. So, it is necessary to produce an {@link FileIO} + * Default factory to provide IO operation interface over files for further transmission them between nodes. + * Some implementations of file senders\receivers are using the zero-copy approach to transfer bytes + * from a file to the given {@link SocketChannel} and vice-versa. So, it is necessary to produce an {@link FileIO} * implementation based on {@link FileChannel} which is reflected in Ignite project as {@link RandomAccessFileIO}. * * @see FileChannel#transferTo(long, long, WritableByteChannel) @@ -294,7 +296,7 @@ public class GridIoManager extends GridManagerAdapter> pairs, SimpleD it.remove(); interruptRecevier(e.getValue(), - new ClusterTopologyCheckedException("Remove node left the grid. " + + new ClusterTopologyCheckedException("Remote node left the grid. " + "Receiver has been stopped : " + nodeId)); } } @@ -1869,7 +1871,7 @@ public boolean checkNodeLeft(UUID nodeId, IgniteCheckedException sndErr, boolean } /** - * @param remoteId The remote note to connect to. + * @param remoteId The remote node to connect to. * @param topic The remote topic to connect to. * @return The channel instance to communicate with remote. */ @@ -1910,7 +1912,7 @@ public void removeTransmissionHandler(Object topic) { * are fully support direct {@link SocketChannel} connection to transfer data. * * @param node Remote node to check. - * @return {@code true} if file can be send over socket channel directly. + * @return {@code true} if a file can be sent over socket channel directly. */ public boolean fileTransmissionSupported(ClusterNode node) { return ((CommunicationSpi)getSpi() instanceof TcpCommunicationSpi) && @@ -1931,7 +1933,7 @@ private IgniteInternalFuture openChannel( ) throws IgniteCheckedException { assert nodeId != null; assert topic != null; - assert !locNodeId.equals(nodeId) : "Channel cannot be opened to the local node itself:" + nodeId; + assert !locNodeId.equals(nodeId) : "Channel cannot be opened to the local node itself: " + nodeId; assert (CommunicationSpi)getSpi() instanceof TcpCommunicationSpi : "Only TcpCommunicationSpi supports direct " + "connections between nodes: " + getSpi().getClass(); @@ -2707,7 +2709,7 @@ private void interruptRecevier(ReceiverContext rctx, Exception ex) { rctx.hnd.onException(rctx.rmtNodeId, ex); - U.error(log, "Receiver has been interrupted due to an excpetion occurred [ctx=" + rctx + ']', ex); + U.error(log, "Receiver has been interrupted due to an exception occurred [ctx=" + rctx + ']', ex); } } @@ -2755,7 +2757,7 @@ private void processOpenedChannel(Object topic, UUID rmtNodeId, SessionChannelMe rcvCtx = rcvCtxs.computeIfAbsent(topic, t -> new ReceiverContext(rmtNodeId, hnd, newSesId)); } - // Do not allow multiple connection for the same session. + // Do not allow multiple connections for the same session. if (!newSesId.equals(rcvCtx.sesId)) { IgniteCheckedException err = new IgniteCheckedException("Requested topic is busy by another transmission. " + "It's not allowed to process different sessions over the same topic simultaneously. " + @@ -2780,7 +2782,7 @@ private void processOpenedChannel(Object topic, UUID rmtNodeId, SessionChannelMe if (rcvCtx.timeoutObj != null) ctx.timeout().removeTimeoutObject(rcvCtx.timeoutObj); - // Send previous context state to sync remote and local node (on manager connected). + // Send previous context state to sync remote and local node. out.writeObject(rcvCtx.lastState == null ? new TransmissionMeta() : rcvCtx.lastState); if (rcvCtx.lastState == null || rcvCtx.lastState.error() == null) @@ -2921,7 +2923,7 @@ private void validate(TransmissionMeta prev, TransmissionMeta next) { /** * @param nodeId Remote node id. - * @param hnd Currnet handler instance which produces file handlers. + * @param hnd Current handler instance which produces file handlers. * @param meta Meta information about file pending to receive to create appropriate receiver. * @param stopChecker Process interrupt checker. * @return Chunk data recevier. @@ -2954,14 +2956,14 @@ private TransmissionReceiver createReceiver( log); default: - throw new IllegalStateException("The type of transmission policy is unknown. An impelentation " + + throw new IllegalStateException("The type of transmission policy is unknown. An implementation " + "required: " + meta.policy()); } } /** * @param channel Socket channel to configure blocking mode. - * @param timeout Ignite network ocnfiguration timeout. + * @param timeout Ignite network configuration timeout. * @throws IOException If fails. */ private static void configureChannel(SocketChannel channel, int timeout) throws IOException { @@ -3043,8 +3045,8 @@ public ReceiverContext(UUID rmtNodeId, TransmissionHandler hnd, IgniteUuid sesId * *

    Zero-copy approach

    *

    - * Current implementation of transmission sender is based on file zero-copy algorithm (the {@link FileSender} - * is used under the hood). It is potentially much more efficient than a simple loop that reads data from + * Current implementation of transmission sender is based on file zero-copy approach (the {@link FileSender} + * is used under the hood). It is much more efficient than a simple loop that reads data from * given file and writes it to the target socket channel. But if operating system does not support zero-copy * file transfer, sending a file with {@link TransmissionSender} might fail or yield worse performance. *

    @@ -3078,13 +3080,13 @@ public ReceiverContext(UUID rmtNodeId, TransmissionHandler hnd, IgniteUuid sesId * All transport level exceptions of transmission file sender will require transmission to be reconnected. * For instance, when the local node closes the socket connection in orderly way, but the file is not fully * handled by remote node, the read operation over the same socket endpoint will return -1. Such - * result will be consideread as an IOException by handler and it will wait for reestablishing connection + * result will be consideread as an {@link IOException} by handler and it will wait for reestablishing connection * to continue file loading. *

    * Another example, the transmission sender gets the Connection reset by peer IOException message. * This means that the remote node you are connected to has to reset the connection. This is usually caused by a * high amount of traffic on the host, but may be caused by a server error or the remote node has exhausted - * system resources as well. Such IOException will be considered as reconnection required. + * system resources as well. Such {@link IOException} will be considered as reconnection required. * *

    Timeout exceptions

    *

    @@ -3092,7 +3094,7 @@ public ReceiverContext(UUID rmtNodeId, TransmissionHandler hnd, IgniteUuid sesId * the {@link Socket#setSoTimeout(int)} will be used and an {@link SocketTimeoutException} will be * thrown when the timeout occured. The default value is taken from {@link IgniteConfiguration#getNetworkTimeout()}. *

    - * If reconnection is not occured withing configured timeout interval the timeout object will be fired which + * If reconnection is not occurred withing configured timeout interval the timeout object will be fired which * clears corresponding to the used topic the {@link ReceiverContext}. * *

    Release resources

    @@ -3118,11 +3120,11 @@ public class TransmissionSender implements Closeable { /** Decorated with data operations socket of output channel. */ private ObjectOutput out; - /** Decoreated with data operations socket of input channel. */ + /** Decorated with data operations socket of input channel. */ private ObjectInput in; /** - * @param rmtId The remote note to connect to. + * @param rmtId The remote node to connect to. * @param topic The remote topic to connect to. */ public TransmissionSender( @@ -3135,7 +3137,7 @@ public TransmissionSender( } /** - * @return The syncronization meta if case connection has been reset. + * @return The synchronization meta if case connection has been reset. * @throws IgniteCheckedException If fails. * @throws IOException If fails. */ @@ -3166,7 +3168,7 @@ private TransmissionMeta connect() throws IgniteCheckedException, IOException { /** * @param file Source file to send to remote. - * @param params Additional transfer file description keys. + * @param params Additional file params. * @param plc The policy of handling data on remote. * @throws IgniteCheckedException If fails. */ @@ -3194,7 +3196,7 @@ public void send( * @param file Source file to send to remote. * @param offset Position to start trasfer at. * @param cnt Number of bytes to transfer. - * @param params Additional transfer file description keys. + * @param params Additional file params. * @param plc The policy of handling data on remote. * @throws IgniteCheckedException If fails. */ @@ -3235,11 +3237,11 @@ public void send( try { TransmissionMeta rcvMeta = null; - // in/out streams are not null if file has been sent successfully + // In/out streams are not null if file has been sent successfully. if (out == null && in == null) { rcvMeta = connect(); - assert rcvMeta != null : "Remote recevier has not sent its meta"; + assert rcvMeta != null : "Remote receiver has not sent its meta"; // Stop in case of any error occurred on remote node during file processing. if (rcvMeta.error() != null) @@ -3251,7 +3253,7 @@ public void send( // Read file received acknowledge. boolean written = in.readBoolean(); - assert written : "File is not fully written :" + file.getAbsolutePath(); + assert written : "File is not fully written: " + file.getAbsolutePath(); break; } @@ -3287,7 +3289,7 @@ public void send( catch (IgniteCheckedException e) { closeChannelQuiet(); - throw new IgniteCheckedException("Excpetion while sending file [rmtId=" + rmtId + + throw new IgniteCheckedException("Exception while sending file [rmtId=" + rmtId + ", file=" + file.getName() + ", sesKey=" + sesKey + ", retries=" + retries + ']', e); } catch (Throwable e) { @@ -3321,7 +3323,7 @@ public void send( out0.flush(); } catch (IOException e) { - U.warn(log, "An excpetion while writing close session flag occured. " + + U.warn(log, "An exception while writing close session flag occured. " + " Session close operation has been ignored", e); } finally { @@ -3329,9 +3331,7 @@ public void send( } } - /** - * Close channel and relese resources. - */ + /** Close channel and relese resources. */ private void closeChannelQuiet() { U.closeQuiet(out); U.closeQuiet(in); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java index 4fb92bcdbd216..7de2a90d1ef00 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/SessionChannelMessage.java @@ -31,7 +31,7 @@ * an appropriate channel is opened. */ class SessionChannelMessage implements Message { - /** Initial channel message type (value is {@code 177}). */ + /** Initial channel message type (value is {@code 175}). */ public static final short TYPE_CODE = 175; /** Serialization version. */ @@ -64,7 +64,7 @@ public IgniteUuid sesId() { /** * @param sesId The unique session id for the channel. - * @return {@code this} for chaining. + * @return {@code This} for chaining. */ public SessionChannelMessage sesId(IgniteUuid sesId) { this.sesId = sesId; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java index 591a54ce6a72f..a55f1e65397ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java @@ -28,16 +28,16 @@ * must be registered to and appropriate topic in {@link GridIoManager} prior to opening a new transmission connection * to this topic. *

    - * NOTE: There is only one such handler per registered topic is allowed for the communication + * NOTE: Only one such handler per registered topic is allowed for the communication * manager. Only one thread is allowed for data processing within a single topic. * *

    TransmissionPolicy

    *

    * Files from the remote node can be handled of two different ways within a single established connection. - * It is up to the sender to decide how the particular file must be prccessed by the remote node. The - * TransmissionPolicy is used for such purpose. If {@link TransmissionPolicy#FILE} type is received by - * remote node the FileHandler will be picked up to process this file, the otherwise for the - * {@link TransmissionPolicy#CHUNK} the ChunkHandler will be picked up. + * It is up to the sender to decide how the particular file must be processed by the remote node. The + * {@link TransmissionPolicy} is used for such purpose. If {@link TransmissionPolicy#FILE} type is received by + * remote node the #fileHandler() will be picked up to process this file, the otherwise for the + * {@link TransmissionPolicy#CHUNK} the #chunkHandler() will be picked up. */ public interface TransmissionHandler { /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java index 6107793b1a66b..0e6dd0e85cfd3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java @@ -40,10 +40,10 @@ class TransmissionMeta implements Externalizable { */ private String name; - /** Offest of transferred file. */ + /** Offset of transferred file. */ private long offset; - /** Number of bytes to transfer started from given offset. */ + /** Number of bytes to transfer started from given {@link #offset}. */ private long cnt; /** Additional file params to transfer (e.g. partition id, partition name etc.). */ @@ -55,9 +55,7 @@ class TransmissionMeta implements Externalizable { /** Last seen error if it has been occurred, or {@code null} the otherwise. */ private Exception err; - /** - * Default constructor, usually used to create meta to read channel data into. - */ + /** Default constructor, usually used to create meta to read channel data into. */ public TransmissionMeta() { this(null); } @@ -70,10 +68,10 @@ public TransmissionMeta(Exception err) { } /** - * @param name The string name representation to assoticate particular meta with. + * @param name File name to assoticate particular meta with. * @param offset The start position of file. * @param cnt Number of bytes expected to transfer. - * @param params Additional transfer meta params. + * @param params Additional meta params. * @param plc Policy of how file will be handled. * @param err Last seen error if it has been occurred, or {@code null} the otherwise. */ @@ -119,14 +117,14 @@ public long count() { } /** - * @return The map of additional keys. + * @return Additional params. */ public Map params() { return params; } /** - * @return File read way policy {@link TransmissionPolicy}. + * @return Transmission policy. */ public TransmissionPolicy policy() { return plc; @@ -134,6 +132,7 @@ public TransmissionPolicy policy() { /** * @param err An exception instance if it has been previously occurred. + * @return {@code This} for chaining. */ public TransmissionMeta error(Exception err) { this.err = err; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionPolicy.java index d4b017a9591c2..be4396f45ef4d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionPolicy.java @@ -20,7 +20,7 @@ import java.util.UUID; /** - * Class represents ways of data handling for a file ready to be sent though an opened transmission sender session. + * Class represents ways of data handling for a file ready to be sent through an opened transmission sender session. * It is necessary to choose which type of handler will be used and how file should be handled prior to sending file * to the remote node. * @@ -28,14 +28,14 @@ */ public enum TransmissionPolicy { /** - * A file which is considered to be sent though TransmissionSenders session will use + * A file which is considered to be sent through {@link GridIoManager.TransmissionSender}s session will use * the {@link TransmissionHandler#fileHandler(UUID, TransmissionMeta)} of {@link TransmissionHandler} * to handle transmitted binary data. */ FILE, /** - * A file which is considered to be sent though TransmissionSenders session will use + * A file which is considered to be sent through {@link GridIoManager.TransmissionSender}s session will use * the {@link TransmissionHandler#chunkHandler(UUID, TransmissionMeta)} of {@link TransmissionHandler} * to handle transmitted binary data. This file will be processed by chunks of handlers defined size. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index 7db6028ea6655..b4254d86b586b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -78,7 +78,7 @@ import static org.apache.ignite.testframework.GridTestUtils.setFieldValue; /** - * Test file transmission mamanger operations. + * Test file transmission manager operations. */ public class GridIoManagerFileTransmissionSelfTest extends GridCommonAbstractTest { /** Number of cache keys to generate. */ @@ -111,9 +111,7 @@ public class GridIoManagerFileTransmissionSelfTest extends GridCommonAbstractTes /** Ignite instance which sends files. */ private IgniteEx snd; - /** - * Called before tests started. - */ + /** Called before tests started. */ @BeforeClass public static void beforeAll() { topic = GridTopic.TOPIC_CACHE.topic("test", 0); @@ -135,9 +133,7 @@ public void before() throws Exception { tempStore = U.resolveWorkDirectory(U.defaultWorkDirectory(), TEMP_FILES_DIR, true); } - /** - * Called after test run. - */ + /** Called after test run. */ @After public void after() { try { @@ -231,11 +227,11 @@ public void testFileHandlerBase() throws Exception { assertEquals(fileSizes.size(), tempStore.listFiles(fileBinFilter).length); for (File file : cacheParts) { - // Check received file lenghs + // Check received file lengths. assertEquals("Received the file length is incorrect: " + file.getName(), fileSizes.get(file.getName()), new Long(file.length())); - // Check received params + // Check received params. assertEquals("File additional parameters are not fully transmitted", fileParams.get(file.getName()), file.hashCode()); } @@ -387,12 +383,12 @@ public void tesFileHandlerReconnectTimeouted() throws Exception { sender.send(fileToSend, TransmissionPolicy.FILE); } catch (IgniteCheckedException | IOException | InterruptedException e) { - // Ignore err + // Ignore err. U.warn(log, e); } assertNotNull("Timeout exception not occurred", refErr.get()); - assertEquals("Type of timeout excpetion incorrect: " + refErr.get(), + assertEquals("Type of timeout exception incorrect: " + refErr.get(), IgniteCheckedException.class, refErr.get().getClass()); } @@ -449,7 +445,7 @@ public void testFileHandlerCleanedUpIfSenderLeft() throws Exception { } assertEquals(NodeStoppingException.class, err.getClass()); - assertEquals("Uncomplete resources must be cleaned up on sender left", + assertEquals("Incomplete resources must be cleaned up on sender left", 0, fileCount(downloadTo.toPath())); } @@ -775,10 +771,10 @@ private void closeIfTransferred() { sender.send(fileToSend, TransmissionPolicy.CHUNK); } - assertEquals("Total number of accepted chunks by remote node is not as expected", + assertEquals("Remote node must accept all chunks", fileToSend.length() / rcv.configuration().getDataStorageConfiguration().getPageSize(), acceptedChunks.get()); - assertEquals("Received file and sent files have not the same lenght", fileToSend.length(), file.length()); + assertEquals("Received file and sent files have not the same lengtgh", fileToSend.length(), file.length()); assertCrcEquals(fileToSend, file); assertNull(fileIo[0]); } @@ -822,7 +818,7 @@ private static void ensureResourcesFree(IgniteEx ig) { ConcurrentMap, AtomicBoolean> sndrFlags = GridTestUtils.getFieldValue(io, "senderStopFlags"); assertTrue("Receiver context map must be empty: " + ctxs, ctxs.isEmpty()); - assertTrue("Sender stop falgs must be empty: " + sndrFlags, sndrFlags.isEmpty()); + assertTrue("Sender stop flags must be empty: " + sndrFlags, sndrFlags.isEmpty()); } /** @@ -839,12 +835,12 @@ private void addCacheData(Ignite ignite, String cacheName) { } /** - * @param ignite The ignite instance. - * @param cacheName Cache name string representation. + * @param ignite An ignite instance. + * @param cacheName Cache name. * @return The cache working directory. */ private File cacheWorkDir(IgniteEx ignite, String cacheName) { - // Resolve cache directory + // Resolve cache directory. IgniteInternalCache cache = ignite.cachex(cacheName); FilePageStoreManager pageStoreMgr = (FilePageStoreManager)cache.context() @@ -894,9 +890,7 @@ private static void assertCrcEquals(File fileToSend, File fileReceived) { } } - /** - * The defailt implementation of transmit session. - */ + /** The defailt implementation of transmit session. */ private static class DefaultTransmissionHandler extends TransmissionHandlerAdapter { /** Ignite recevier node. */ private final IgniteEx rcv; From 8e3886d031ce58fdeb678ac7f19139b5924758d4 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 12 Aug 2019 17:23:29 +0300 Subject: [PATCH 66/69] IGNITE-10619: revert set listener communication spi --- .../spi/communication/tcp/TcpCommunicationSpi.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index e68376d1a41a8..569798eddf07b 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -1255,7 +1255,7 @@ class ConnectClosure implements IgniteInClosure { private final ConcurrentMap clients = GridConcurrentFactory.newMap(); /** SPI listener. */ - private volatile CommunicationListenerEx lsnr; + private volatile CommunicationListener lsnr; /** Bound port. */ private int boundTcpPort = -1; @@ -1968,7 +1968,7 @@ public TcpCommunicationSpi setSlowClientQueueLimit(int slowClientQueueLimit) { /** {@inheritDoc} */ @Override public void setListener(CommunicationListener lsnr) { - this.lsnr = (CommunicationListenerEx)lsnr; + this.lsnr = lsnr; } /** @@ -4048,10 +4048,10 @@ private void notifyChannelEvtListener(UUID nodeId, Channel channel, Message init if (log.isDebugEnabled()) log.debug("Notify appropriate listeners due to a new channel opened: " + channel); - CommunicationListenerEx lsnr0 = lsnr; + CommunicationListener lsnr0 = lsnr; - if (lsnr0 != null) - lsnr0.onChannelOpened(nodeId, initMsg, channel); + if (lsnr0 instanceof CommunicationListenerEx) + ((CommunicationListenerEx)lsnr0).onChannelOpened(nodeId, initMsg, channel); } /** From 32520944d59148a6d2deed427e0c3fb6b8bae13c Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 12 Aug 2019 17:25:56 +0300 Subject: [PATCH 67/69] IGNITE-10619: revert set listener communication spi 2 --- .../spi/communication/GridAbstractCommunicationSelfTest.java | 3 +-- .../tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java | 4 ++-- .../communication/tcp/GridTcpCommunicationSpiLanLoadTest.java | 4 ++-- .../tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java | 4 ++-- .../tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java | 4 ++-- .../tcp/GridTcpCommunicationSpiRecoverySelfTest.java | 4 ++-- .../tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java | 4 ++-- 7 files changed, 13 insertions(+), 14 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java index d468489198ccf..8034093c3cc44 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java @@ -37,7 +37,6 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; -import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -90,7 +89,7 @@ public abstract class GridAbstractCommunicationSelfTest { + private class MessageListener implements CommunicationListener { /** */ private final UUID locNodeId; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java index 4fbda4e99e02d..7ab0d6fb819f8 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java @@ -48,9 +48,9 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; +import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -118,7 +118,7 @@ public GridTcpCommunicationSpiConcurrentConnectSelfTest() { /** * */ - private static class MessageListener implements CommunicationListenerEx { + private static class MessageListener implements CommunicationListener { /** */ private final CountDownLatch latch; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java index bd50c5ba167ff..b4bc2e9c3dd90 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanLoadTest.java @@ -33,8 +33,8 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -86,7 +86,7 @@ public GridTcpCommunicationSpiLanLoadTest() { /** * Accumulating listener. */ - private class MessageListener implements CommunicationListenerEx { + private class MessageListener implements CommunicationListener { /** Node id of local node. */ private final UUID locNodeId; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java index fc756ff25ba7f..cfde86b318f74 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java @@ -52,9 +52,9 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; +import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -126,7 +126,7 @@ public GridTcpCommunicationSpiMultithreadedSelfTest() { /** * Accumulating listener. */ - private static class MessageListener implements CommunicationListenerEx { + private static class MessageListener implements CommunicationListener { /** Node id of local node. */ private final UUID locNodeId; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java index c879114b9b385..408eb1066d526 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java @@ -40,9 +40,9 @@ import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -91,7 +91,7 @@ public GridTcpCommunicationSpiRecoveryAckSelfTest() { } /** */ - private class TestListener implements CommunicationListenerEx { + private class TestListener implements CommunicationListener { /** */ private GridConcurrentHashSet msgIds = new GridConcurrentHashSet<>(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java index f306b1e5ce74d..5ec734a017c7c 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java @@ -44,9 +44,9 @@ import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -105,7 +105,7 @@ public GridTcpCommunicationSpiRecoverySelfTest() { } /** */ - private class TestListener implements CommunicationListenerEx { + private class TestListener implements CommunicationListener { /** */ private boolean block; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java index b145b4a091402..d937bb01ee1cd 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java @@ -43,9 +43,9 @@ import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridTestMessage; -import org.apache.ignite.spi.communication.tcp.internal.CommunicationListenerEx; import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; @@ -95,7 +95,7 @@ public IgniteTcpCommunicationRecoveryAckClosureSelfTest() { } /** */ - private class TestListener implements CommunicationListenerEx { + private class TestListener implements CommunicationListener { /** */ private GridConcurrentHashSet msgIds = new GridConcurrentHashSet<>(); From 49ee526233d420fa172d3ae5bd1253f763213fa7 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 12 Aug 2019 17:30:58 +0300 Subject: [PATCH 68/69] IGNITE-10619: javadoc fix 3 --- .../internal/managers/communication/TransmissionMeta.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java index 0e6dd0e85cfd3..986bf555565eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java @@ -94,7 +94,7 @@ public TransmissionMeta( } /** - * @return String representation file name. + * @return File name. */ public String name() { assert name != null; From f3df4723b5360bb264be4dc8e1c7cd3fe26e248d Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 12 Aug 2019 18:44:33 +0300 Subject: [PATCH 69/69] IGNITE-10619: fix Ignite features --- .../java/org/apache/ignite/internal/IgniteFeatures.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java index b7b8521377b91..f356a9be1e700 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java @@ -63,12 +63,12 @@ public enum IgniteFeatures { /** Support of suspend/resume operations for pessimistic transactions. */ SUSPEND_RESUME_PESSIMISTIC_TX(10), - /** The node can communicate with others via socket channel. */ - CHANNEL_COMMUNICATION(9), - /** Distributed metastorage. */ DISTRIBUTED_METASTORAGE(11), + /** The node can communicate with others via socket channel. */ + CHANNEL_COMMUNICATION(12), + /** Replacing TcpDiscoveryNode field with nodeId field in discovery messages. */ TCP_DISCOVERY_MESSAGE_NODE_COMPACT_REPRESENTATION(14);