HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via umamahesh)
This commit is contained in:
parent
576b569b6c
commit
796a676d18
|
@ -1737,6 +1737,8 @@ Release 2.8.0 - UNRELEASED
|
|||
HDFS-9472. concat() API does not give proper exception messages on ./reserved
|
||||
relative path (Rakesh R via umamahesh)
|
||||
|
||||
HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via umamahesh)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
|
||||
|
|
|
@ -1072,27 +1072,17 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
public void verifyReplication(String src,
|
||||
short replication,
|
||||
String clientName) throws IOException {
|
||||
String err = null;
|
||||
if (replication > maxReplication) {
|
||||
err = " exceeds maximum of " + maxReplication;
|
||||
} else if (replication < minReplication) {
|
||||
err = " is less than the required minimum of " + minReplication;
|
||||
}
|
||||
|
||||
if (replication < minReplication || replication > maxReplication) {
|
||||
StringBuilder msg = new StringBuilder("Requested replication factor of ");
|
||||
|
||||
msg.append(replication);
|
||||
|
||||
if (replication > maxReplication) {
|
||||
msg.append(" exceeds maximum of ");
|
||||
msg.append(maxReplication);
|
||||
} else {
|
||||
msg.append(" is less than the required minimum of ");
|
||||
msg.append(minReplication);
|
||||
}
|
||||
|
||||
msg.append(" for ").append(src);
|
||||
|
||||
if (clientName != null) {
|
||||
msg.append(" from ").append(clientName);
|
||||
}
|
||||
|
||||
throw new IOException(msg.toString());
|
||||
if (err != null) {
|
||||
throw new IOException("Requested replication factor of " + replication
|
||||
+ err + " for " + src
|
||||
+ (clientName == null? "": ", clientName=" + clientName));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1228,8 +1218,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
}
|
||||
}
|
||||
if (datanodes.length() != 0) {
|
||||
blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock,
|
||||
datanodes.toString());
|
||||
blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock, datanodes);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2850,7 +2839,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
if (result == AddBlockResult.ADDED) {
|
||||
curReplicaDelta = 1;
|
||||
if (logEveryBlock) {
|
||||
logAddStoredBlock(storedBlock, node);
|
||||
blockLog.debug("BLOCK* addStoredBlock: {} is added to {} (size={})",
|
||||
node, storedBlock, storedBlock.getNumBytes());
|
||||
}
|
||||
} else if (result == AddBlockResult.REPLACED) {
|
||||
curReplicaDelta = 0;
|
||||
|
@ -2923,21 +2913,6 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
return storedBlock;
|
||||
}
|
||||
|
||||
private void logAddStoredBlock(BlockInfo storedBlock,
|
||||
DatanodeDescriptor node) {
|
||||
if (!blockLog.isDebugEnabled()) {
|
||||
return;
|
||||
}
|
||||
|
||||
StringBuilder sb = new StringBuilder(500);
|
||||
sb.append("BLOCK* addStoredBlock: blockMap updated: ")
|
||||
.append(node)
|
||||
.append(" is added to ");
|
||||
storedBlock.appendStringTo(sb);
|
||||
sb.append(" size " )
|
||||
.append(storedBlock.getNumBytes());
|
||||
blockLog.debug(sb.toString());
|
||||
}
|
||||
/**
|
||||
* Invalidate corrupt replicas.
|
||||
* <p>
|
||||
|
|
|
@ -17,6 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.server.blockmanagement;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
import static org.apache.hadoop.util.Time.monotonicNow;
|
||||
|
||||
import java.util.AbstractList;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
|
@ -26,13 +29,9 @@ import java.util.Queue;
|
|||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
|
@ -44,8 +43,9 @@ import org.apache.hadoop.util.ChunkedArrayList;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
import static org.apache.hadoop.util.Time.monotonicNow;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* Manages datanode decommissioning. A background monitor thread
|
||||
|
@ -441,17 +441,10 @@ public class DecommissionManager {
|
|||
LOG.debug("Node {} is sufficiently replicated and healthy, "
|
||||
+ "marked as decommissioned.", dn);
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
StringBuilder b = new StringBuilder("Node {} ");
|
||||
if (isHealthy) {
|
||||
b.append("is ");
|
||||
} else {
|
||||
b.append("isn't ");
|
||||
}
|
||||
b.append("healthy and still needs to replicate {} more blocks," +
|
||||
" decommissioning is still in progress.");
|
||||
LOG.debug(b.toString(), dn, blocks.size());
|
||||
}
|
||||
LOG.debug("Node {} {} healthy."
|
||||
+ " It needs to replicate {} more blocks."
|
||||
+ " Decommissioning is still in progress.",
|
||||
dn, isHealthy? "is": "isn't", blocks.size());
|
||||
}
|
||||
} else {
|
||||
LOG.debug("Node {} still has {} blocks to replicate "
|
||||
|
@ -587,8 +580,7 @@ public class DecommissionManager {
|
|||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void runMonitor() throws ExecutionException, InterruptedException {
|
||||
Future f = executor.submit(monitor);
|
||||
f.get();
|
||||
void runMonitorForTest() throws ExecutionException, InterruptedException {
|
||||
executor.submit(monitor).get();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -317,19 +317,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
NameNodeMXBean {
|
||||
public static final Log LOG = LogFactory.getLog(FSNamesystem.class);
|
||||
|
||||
private static final ThreadLocal<StringBuilder> auditBuffer =
|
||||
new ThreadLocal<StringBuilder>() {
|
||||
@Override
|
||||
protected StringBuilder initialValue() {
|
||||
return new StringBuilder();
|
||||
}
|
||||
};
|
||||
|
||||
private final BlockIdManager blockIdManager;
|
||||
|
||||
@VisibleForTesting
|
||||
public boolean isAuditEnabled() {
|
||||
return !isDefaultAuditLogger || auditLog.isInfoEnabled();
|
||||
boolean isAuditEnabled() {
|
||||
return (!isDefaultAuditLogger || auditLog.isInfoEnabled())
|
||||
&& !auditLoggers.isEmpty();
|
||||
}
|
||||
|
||||
private void logAuditEvent(boolean succeeded, String cmd, String src)
|
||||
|
@ -358,14 +350,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
stat.getAccessTime(), stat.getPermission(), stat.getOwner(),
|
||||
stat.getGroup(), symlink, path);
|
||||
}
|
||||
final String ugiStr = ugi.toString();
|
||||
for (AuditLogger logger : auditLoggers) {
|
||||
if (logger instanceof HdfsAuditLogger) {
|
||||
HdfsAuditLogger hdfsLogger = (HdfsAuditLogger) logger;
|
||||
hdfsLogger.logAuditEvent(succeeded, ugi.toString(), addr, cmd, src, dst,
|
||||
hdfsLogger.logAuditEvent(succeeded, ugiStr, addr, cmd, src, dst,
|
||||
status, CallerContext.getCurrent(), ugi, dtSecretManager);
|
||||
} else {
|
||||
logger.logAuditEvent(succeeded, ugi.toString(), addr,
|
||||
cmd, src, dst, status);
|
||||
logger.logAuditEvent(succeeded, ugiStr, addr, cmd, src, dst, status);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -616,15 +608,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
// sharedEditsDirs.
|
||||
if (!editsDirs.contains(u) &&
|
||||
!sharedEditsDirs.contains(u)) {
|
||||
throw new IllegalArgumentException(
|
||||
"Required edits directory " + u.toString() + " not present in " +
|
||||
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + ". " +
|
||||
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + "=" +
|
||||
editsDirs.toString() + "; " +
|
||||
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY + "=" +
|
||||
requiredEditsDirs.toString() + ". " +
|
||||
DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY + "=" +
|
||||
sharedEditsDirs.toString() + ".");
|
||||
throw new IllegalArgumentException("Required edits directory " + u
|
||||
+ " not found: "
|
||||
+ DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + "=" + editsDirs + "; "
|
||||
+ DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY
|
||||
+ "=" + requiredEditsDirs + "; "
|
||||
+ DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY
|
||||
+ "=" + sharedEditsDirs);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -698,11 +688,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache)
|
||||
throws IOException {
|
||||
provider = DFSUtil.createKeyProviderCryptoExtension(conf);
|
||||
if (provider == null) {
|
||||
LOG.info("No KeyProvider found.");
|
||||
} else {
|
||||
LOG.info("Found KeyProvider: " + provider.toString());
|
||||
}
|
||||
LOG.info("KeyProvider: " + provider);
|
||||
if (conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY,
|
||||
DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT)) {
|
||||
LOG.info("Enabling async auditlog");
|
||||
|
@ -2077,11 +2063,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
.append(", clientMachine=").append(clientMachine)
|
||||
.append(", createParent=").append(createParent)
|
||||
.append(", replication=").append(replication)
|
||||
.append(", createFlag=").append(flag.toString())
|
||||
.append(", createFlag=").append(flag)
|
||||
.append(", blockSize=").append(blockSize)
|
||||
.append(", supportedVersions=")
|
||||
.append(supportedVersions == null ? null : Arrays.toString
|
||||
(supportedVersions));
|
||||
.append(Arrays.toString(supportedVersions));
|
||||
NameNode.stateChangeLog.debug(builder.toString());
|
||||
}
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
|
@ -2513,43 +2498,40 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
getEditLog().logSync();
|
||||
}
|
||||
|
||||
INodeFile checkLease(
|
||||
String src, String holder, INode inode, long fileId)
|
||||
private String leaseExceptionString(String src, long fileId, String holder) {
|
||||
final Lease lease = leaseManager.getLease(holder);
|
||||
return src + " (inode " + fileId + ") " + (lease != null? lease.toString()
|
||||
: "Holder " + holder + " does not have any open files.");
|
||||
}
|
||||
|
||||
INodeFile checkLease(String src, String holder, INode inode, long fileId)
|
||||
throws LeaseExpiredException, FileNotFoundException {
|
||||
assert hasReadLock();
|
||||
final String ident = src + " (inode " + fileId + ")";
|
||||
if (inode == null) {
|
||||
Lease lease = leaseManager.getLease(holder);
|
||||
throw new FileNotFoundException(
|
||||
"No lease on " + ident + ": File does not exist. "
|
||||
+ (lease != null ? lease.toString()
|
||||
: "Holder " + holder + " does not have any open files."));
|
||||
throw new FileNotFoundException("File does not exist: "
|
||||
+ leaseExceptionString(src, fileId, holder));
|
||||
}
|
||||
if (!inode.isFile()) {
|
||||
Lease lease = leaseManager.getLease(holder);
|
||||
throw new LeaseExpiredException(
|
||||
"No lease on " + ident + ": INode is not a regular file. "
|
||||
+ (lease != null ? lease.toString()
|
||||
: "Holder " + holder + " does not have any open files."));
|
||||
throw new LeaseExpiredException("INode is not a regular file: "
|
||||
+ leaseExceptionString(src, fileId, holder));
|
||||
}
|
||||
final INodeFile file = inode.asFile();
|
||||
if (!file.isUnderConstruction()) {
|
||||
Lease lease = leaseManager.getLease(holder);
|
||||
throw new LeaseExpiredException(
|
||||
"No lease on " + ident + ": File is not open for writing. "
|
||||
+ (lease != null ? lease.toString()
|
||||
: "Holder " + holder + " does not have any open files."));
|
||||
throw new LeaseExpiredException("File is not open for writing: "
|
||||
+ leaseExceptionString(src, fileId, holder));
|
||||
}
|
||||
// No further modification is allowed on a deleted file.
|
||||
// A file is considered deleted, if it is not in the inodeMap or is marked
|
||||
// as deleted in the snapshot feature.
|
||||
if (isFileDeleted(file)) {
|
||||
throw new FileNotFoundException(src);
|
||||
throw new FileNotFoundException("File is deleted: "
|
||||
+ leaseExceptionString(src, fileId, holder));
|
||||
}
|
||||
String clientName = file.getFileUnderConstructionFeature().getClientName();
|
||||
if (holder != null && !clientName.equals(holder)) {
|
||||
throw new LeaseExpiredException("Lease mismatch on " + ident +
|
||||
" owned by " + clientName + " but is accessed by " + holder);
|
||||
final String owner = file.getFileUnderConstructionFeature().getClientName();
|
||||
if (holder != null && !owner.equals(holder)) {
|
||||
throw new LeaseExpiredException("Client (=" + holder
|
||||
+ ") is not the lease owner (=" + owner + ": "
|
||||
+ leaseExceptionString(src, fileId, holder));
|
||||
}
|
||||
return file;
|
||||
}
|
||||
|
@ -6188,7 +6170,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
if (success) {
|
||||
getEditLog().logSync();
|
||||
}
|
||||
String idStr = "{id: " + directive.getId().toString() + "}";
|
||||
final String idStr = "{id: " + directive.getId() + "}";
|
||||
logAuditEvent(success, "modifyCacheDirective", idStr,
|
||||
directive.toString(), null);
|
||||
}
|
||||
|
@ -6662,6 +6644,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
*/
|
||||
@VisibleForTesting
|
||||
static class DefaultAuditLogger extends HdfsAuditLogger {
|
||||
private static final ThreadLocal<StringBuilder> STRING_BUILDER =
|
||||
new ThreadLocal<StringBuilder>() {
|
||||
@Override
|
||||
protected StringBuilder initialValue() {
|
||||
return new StringBuilder();
|
||||
}
|
||||
};
|
||||
|
||||
private boolean isCallerContextEnabled;
|
||||
private int callerContextMaxLen;
|
||||
private int callerSignatureMaxLen;
|
||||
|
@ -6696,7 +6686,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
|
||||
if (auditLog.isDebugEnabled() ||
|
||||
(auditLog.isInfoEnabled() && !debugCmdSet.contains(cmd))) {
|
||||
final StringBuilder sb = auditBuffer.get();
|
||||
final StringBuilder sb = STRING_BUILDER.get();
|
||||
sb.setLength(0);
|
||||
sb.append("allowed=").append(succeeded).append("\t");
|
||||
sb.append("ugi=").append(userName).append("\t");
|
||||
|
|
|
@ -320,7 +320,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|||
sb.append("FSCK started by " +
|
||||
UserGroupInformation.getCurrentUser() + " from " +
|
||||
remoteAddress + " at " + new Date());
|
||||
out.println(sb.toString());
|
||||
out.println(sb);
|
||||
sb.append(" for blockIds: \n");
|
||||
for (String blk: blocks) {
|
||||
if(blk == null || !blk.contains(Block.BLOCK_FILE_PREFIX)) {
|
||||
|
@ -331,7 +331,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|||
blockIdCK(blk);
|
||||
sb.append(blk + "\n");
|
||||
}
|
||||
LOG.info(sb.toString());
|
||||
LOG.info(sb);
|
||||
namenode.getNamesystem().logFsckEvent("/", remoteAddress);
|
||||
out.flush();
|
||||
return;
|
||||
|
@ -383,7 +383,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|||
out.println(ecRes);
|
||||
|
||||
if (this.showStoragePolcies) {
|
||||
out.print(storageTypeSummary.toString());
|
||||
out.print(storageTypeSummary);
|
||||
}
|
||||
|
||||
out.println("FSCK ended at " + new Date() + " in "
|
||||
|
@ -725,7 +725,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|||
// might still be accessible as the block might be incorrectly marked as
|
||||
// corrupted by client machines.
|
||||
report.append(" MISSING!");
|
||||
res.addMissing(block.toString(), block.getNumBytes());
|
||||
res.addMissing(blkName, block.getNumBytes());
|
||||
missing++;
|
||||
missize += block.getNumBytes();
|
||||
} else {
|
||||
|
@ -793,7 +793,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|||
out.print(" OK\n");
|
||||
}
|
||||
if (showBlocks) {
|
||||
out.print(report.toString() + "\n");
|
||||
out.print(report + "\n");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|||
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
||||
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.log4j.Level;
|
||||
|
@ -198,14 +199,12 @@ public class TestFileAppend4 {
|
|||
t.join();
|
||||
LOG.info("Close finished.");
|
||||
|
||||
// We expect that close will get a "File is not open"
|
||||
// error.
|
||||
// We expect that close will get a "File is not open" error.
|
||||
Throwable thrownByClose = err.get();
|
||||
assertNotNull(thrownByClose);
|
||||
assertTrue(thrownByClose instanceof IOException);
|
||||
if (!thrownByClose.getMessage().contains(
|
||||
"No lease on /testRecoverFinalized"))
|
||||
throw thrownByClose;
|
||||
assertTrue(thrownByClose instanceof LeaseExpiredException);
|
||||
GenericTestUtils.assertExceptionContains("File is not open for writing",
|
||||
thrownByClose);
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -281,10 +280,9 @@ public class TestFileAppend4 {
|
|||
// error.
|
||||
Throwable thrownByClose = err.get();
|
||||
assertNotNull(thrownByClose);
|
||||
assertTrue(thrownByClose instanceof IOException);
|
||||
if (!thrownByClose.getMessage().contains(
|
||||
"Lease mismatch"))
|
||||
throw thrownByClose;
|
||||
assertTrue(thrownByClose instanceof LeaseExpiredException);
|
||||
GenericTestUtils.assertExceptionContains("not the lease owner",
|
||||
thrownByClose);
|
||||
|
||||
// The appender should be able to close properly
|
||||
appenderStream.close();
|
||||
|
|
|
@ -165,7 +165,7 @@ public class TestFileCreation {
|
|||
cluster.waitActive();
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
try {
|
||||
FsServerDefaults serverDefaults = fs.getServerDefaults();
|
||||
FsServerDefaults serverDefaults = fs.getServerDefaults(new Path("/"));
|
||||
assertEquals(DFS_BLOCK_SIZE_DEFAULT, serverDefaults.getBlockSize());
|
||||
assertEquals(DFS_BYTES_PER_CHECKSUM_DEFAULT, serverDefaults.getBytesPerChecksum());
|
||||
assertEquals(DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT, serverDefaults.getWritePacketSize());
|
||||
|
@ -413,8 +413,7 @@ public class TestFileCreation {
|
|||
stm1.close();
|
||||
fail("Should have exception closing stm1 since it was deleted");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("No lease on /testfile", ioe);
|
||||
GenericTestUtils.assertExceptionContains("File does not exist.", ioe);
|
||||
GenericTestUtils.assertExceptionContains("File does not exist", ioe);
|
||||
}
|
||||
|
||||
} finally {
|
||||
|
@ -805,7 +804,6 @@ public class TestFileCreation {
|
|||
public static void testFileCreationNonRecursive(FileSystem fs) throws IOException {
|
||||
final Path path = new Path("/" + Time.now()
|
||||
+ "-testFileCreationNonRecursive");
|
||||
FSDataOutputStream out = null;
|
||||
IOException expectedException = null;
|
||||
final String nonExistDir = "/non-exist-" + Time.now();
|
||||
|
||||
|
|
|
@ -306,7 +306,7 @@ public class BlockManagerTestUtil {
|
|||
*/
|
||||
public static void recheckDecommissionState(DatanodeManager dm)
|
||||
throws ExecutionException, InterruptedException {
|
||||
dm.getDecomManager().runMonitor();
|
||||
dm.getDecomManager().runMonitorForTest();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue