文章目录

  • 前言
  • HDFS Multiple Standby的实现要素
  • Multiple Standby实现分析
    • Bootstrap行为
    • Standby NN的checkpoint upload行为
    • Active/Standby NN间的failover行为
    • Standby NN向ActiveNN发起的log roll操作行为
  • 相关链接

前言


HDFS在早期实现HA时,是标准的一主一备的服务模式,主的叫Active NameNode,备的叫Standby NameNode。Standby/Active NN间可以互相切换以此达到服务高可用的目的。但是这种双节点的HA模式是否能够满足更高的高可用性的要求呢?在标准的HA模式下,其实只有1个Standby的NN作为bak来使用。假设在极端情况下,Active和Stanby同时出现crash的情况(这种概率在实际生产环境中也是有几率发生的),那么此时HDFS集群将处于不能服务的状态。因此,我们是否能够实现一种具有更高HA特性的服务模式呢,比如1个Active Service+多Standby的运行模式?倘若我们有了多Standby的支持,毫无疑问将会大大提高NN服务的高可用性。目前社区在3.x版本中已经实现了此功能,相关JIRA HDFS-6440(Support more than 2 NameNodes)。本文笔者对此功能做一个简单的原理分析,在了解了其原理实现后,能够帮助我们更好地去使用这个功能特性。

HDFS Multiple Standby的实现要素


首先我们要知道实现HDFS Multiple Standby的前提是基于HDFS原有的HA实现之上的。它只是将原有的Active-Standby模式扩展为Active-多Standby模式。

从Single Standby到Multiple Standby的关系转变上,有一个核心的点是不变的:Active和Standby服务之间的交互行为。简单来说,就是原本Active和Standby NN服务之间的交互通信理应是不变的。在HA核心代码实现上我们并不需要做额外的改动。

因此在这里面,我们要重点关注以下Active、Standby NN之间的交互行为:

  • NN Bootstrap行为
  • Standby NN的checkpoint upload到Active NN的行为
  • Active/Standby NN间的failover行为
  • Standby NN向ActiveNN发起的log roll操作行为

上述四类操作在原有1对1的Active/Standby模式时,是比较简单的单向直接调用行为。但是在多Standby时,这个时候target会变成多个,而且这些 target的状态事先也不确定,可能是Standby也可能是Active。这部分也是多Standby实现的一个重点也是难点。

下面我们结合实际场景代码做逐一分析。

Multiple Standby实现分析


此小节部分我们主要对上面提到的4个交互行为做具体分析。

Bootstrap行为


在单Standby模式里,Standby NN的直接向另外一个Active NN发起download image的行为即可。但是在多Standby时,操作行为转变为:只需寻找到一个正在服务的NN进行bootstrp即可,不管这个NN是真正的Active还是Standby。因为bootstrap行为只是去NN的metadata文件数据。

相关代码如下:
BootstrapStandby.java

  private int doRun() throws IOException {// find the active NNNamenodeProtocol proxy = null;NamespaceInfo nsInfo = null;boolean isUpgradeFinalized = false;RemoteNameNodeInfo proxyInfo = null;for (int i = 0; i < remoteNNs.size(); i++) {proxyInfo = remoteNNs.get(i);InetSocketAddress otherIpcAddress = proxyInfo.getIpcAddress();proxy = createNNProtocolProxy(otherIpcAddress);try {// Get the namespace from any active NN. If you just formatted the primary NN and are// bootstrapping the other NNs from that layout, it will only contact the single NN.// However, if there cluster is already running and you are adding a NN later (e.g.// replacing a failed NN), then this will bootstrap from any node in the cluster.nsInfo = proxy.versionRequest();isUpgradeFinalized = proxy.isUpgradeFinalized();// 寻找到一个可用的NN即可break;} catch (IOException ioe) {LOG.warn("Unable to fetch namespace information from remote NN at " + otherIpcAddress+ ": " + ioe.getMessage());if (LOG.isDebugEnabled()) {LOG.debug("Full exception trace", ioe);}}}...// download the fsimage from active namenodeint download = downloadImage(storage, proxy, proxyInfo);if (download != 0) {return download;}// finish the upgrade: rename previous.tmp to previous...return 0;}

Standby NN的checkpoint upload行为


我们知道在HDFS HA机制里,Standby会定期checkpoint出一个新的image,然后upload到Active NN中。那么在多Standby NN的情况下,需要解决下面2点主要问题:

  • Standby NN如何找到谁是真正的Active NN,然后再进行image的upload。
  • 找到谁是真正的Active NN后,各个Standby如何协调进行image的upload,这里面会存在潜在的冲突问题。

对于第一个问题,社区的解决思路是假定所有的NN都是潜在的Active NN(里面肯定包括Standby NN)。然后检查每个NN上次发送成功image的状态信息,这个状态信息主要记录以下2点信息:

1)是否是Active节点,这个是基于上次image upload结果发现的,如果后面发生了failover,这个属性值会被更新。
2)上次image upload成功后的时间。

检查的条件:要么target NN是Active节点,要么或者是距上次image upload时间超过指定阈值范围内的即可。相关代码如下:

StandbyCheckpointer.java

  private void doCheckpoint() throws InterruptedException, IOException {assert canceler != null;final long txid;final NameNodeFile imageType;...// checkpoint操作生成新的image文件img.saveNamespace(namesystem, imageType, canceler);txid = img.getStorage().getMostRecentCheckpointTxId();assert txid == thisCheckpointTxId : "expected to save checkpoint at txid=" +thisCheckpointTxId + " but instead saved at txid=" + txid;...// Upload the saved checkpoint back to the active// Do this in a separate thread to avoid blocking transition to active, but don't allow more// than the expected number of tasks to run or queue up// See HDFS-4816ExecutorService executor = new ThreadPoolExecutor(0, activeNNAddresses.size(), 100,TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(activeNNAddresses.size()),uploadThreadFactory);// for right now, just match the upload to the nn address by convention. There is no need to// directly tie them together by adding a pair class.HashMap<String, Future<TransferFsImage.TransferResult>> uploads =new HashMap<>();for (final URL activeNNAddress : activeNNAddresses) {// Upload image if at least 1 of 2 following conditions met:// 1. has been quiet for long enough, try to contact the node.// 2. this standby IS the primary checkpointer of target NN.String addressString = activeNNAddress.toString();assert checkpointReceivers.containsKey(addressString);// 1)获取目标NN上次checkpoint的状态CheckpointReceiverEntry receiverEntry =checkpointReceivers.get(addressString);long secsSinceLastUpload =TimeUnit.MILLISECONDS.toSeconds(monotonicNow() - receiverEntry.getLastUploadTime());// 2)如果目标NN是Active或者上次checkpoint时间超出阈值外,则应该upload checkpoint到此NNboolean shouldUpload = receiverEntry.isPrimary() ||secsSinceLastUpload >= checkpointConf.getQuietPeriod();if (shouldUpload) {// 3)提交执行upload image到指定NN的任务Future<TransferFsImage.TransferResult> upload =executor.submit(new Callable<TransferFsImage.TransferResult>() {@Overridepublic TransferFsImage.TransferResult call()throws IOException, InterruptedException {CheckpointFaultInjector.getInstance().duringUploadInProgess();return TransferFsImage.uploadImageFromStorage(activeNNAddress,conf, namesystem.getFSImage().getStorage(), imageType, txid,canceler);}});uploads.put(addressString, upload);}}
...
}

然后在获取upload结果时进行相应target NN的receive结果更新。

    for (Map.Entry<String, Future<TransferFsImage.TransferResult>> entry :uploads.entrySet()) {String url = entry.getKey();Future<TransferFsImage.TransferResult> upload = entry.getValue();try {// TODO should there be some smarts here about retries nodes that//  are not the active NN?CheckpointReceiverEntry receiverEntry = checkpointReceivers.get(url);TransferFsImage.TransferResult uploadResult = upload.get();// 4.a)获取image upload结果,如果执行成功更新对应checkpoint receiver entry的状态://  1)标记receiver的Primary为true,意为此为当前真正的Active的节点 2)更新时间if (uploadResult == TransferFsImage.TransferResult.SUCCESS) {receiverEntry.setLastUploadTime(monotonicNow());receiverEntry.setIsPrimary(true);} else {// Getting here means image upload is explicitly rejected// by the other node. This could happen if:// 1. the other is also a standby, or// 2. the other is active, but already accepted another// newer image, or// 3. the other is active but has a recent enough image.// All these are valid cases, just log for information.LOG.info("Image upload rejected by the other NameNode: {}",uploadResult);// 4.b)否则将receiver的primary属性设置为falsereceiverEntry.setIsPrimary(false);}} catch (ExecutionException e) {// Even if exception happens, still proceeds to next NN url.// so that fail to upload to previous NN does not cause the// remaining NN not getting the fsImage.ioes.add(new IOException("Exception during image upload", e));} catch (InterruptedException e) {ie = e;break;}}

从上面代码我们可以看到,这里对每个target NN更新了最新receive image的结果,同时也能够识别出哪个是真正的Active节点(通过Primary属性)。因为还是存在有NN failover到其它NN的可能性,所以这里Standby NN还是向所有的其它NN发起了image的upload请求。如果target是Standby NN的话,这里会接受到一个错误的返回结果,也就实际不会发生upload image到Standby的情况了。

如果target NN是真正的Active NN,那么是否可能会发生多Standby NN频繁upload image到Active NN的情况呢?毕竟每个Standby NN都会做这样的checkpoint逻辑。要解决这个问题,我们需要在Active NN接收方这边做一些额外的判断了。

相关代码如下:
ImageServlet.java

  protected void doPut(final HttpServletRequest request,final HttpServletResponse response) throws ServletException, IOException {try {...UserGroupInformation.getCurrentUser().doAs(new PrivilegedExceptionAction<Void>() {@Overridepublic Void run() throws Exception {// 1) 获取当前NN状态,判断是否是Active的NN,这里target的目标是Active NN//  如果当前是Standby NN收到请求了,则返回的是错误结果HAServiceProtocol.HAServiceState state = NameNodeHttpServer.getNameNodeStateFromContext(getServletContext());if (state != HAServiceProtocol.HAServiceState.ACTIVE &&state != HAServiceProtocol.HAServiceState.OBSERVER) {// we need a different response type here so the client can differentiate this// from the failure to upload due to (1) security, or (2) other checkpoints already// presentsendError(response, HttpServletResponse.SC_EXPECTATION_FAILED,"Nameode "+request.getLocalAddr()+" is currently not in a state which can "+ "accept uploads of new fsimages. State: "+state);return null;}final long txid = parsedParams.getTxId();String remoteAddr = request.getRemoteAddr();ImageUploadRequest imageRequest = new ImageUploadRequest(txid, remoteAddr);final NameNodeFile nnf = parsedParams.getNameNodeFile();// 2)判断当前是否有别的upload image正在被执行SortedSet<ImageUploadRequest> larger = currentlyDownloadingCheckpoints.tailSet(imageRequest);if (larger.size() > 0) {sendError(response, HttpServletResponse.SC_CONFLICT,"Another checkpointer is already in the process of uploading a" +" checkpoint made up to transaction ID " + larger.last());return null;}//make sure no one else has started uploading oneif (!currentlyDownloadingCheckpoints.add(imageRequest)) {sendError(response, HttpServletResponse.SC_CONFLICT,"Either current namenode is checkpointing or another"+ " checkpointer is already in the process of "+ "uploading a checkpoint made at transaction ID "+ txid);return null;}...// 3) 计算NN上次checkpoint距离目前的时间间隔以及Transaction数的差值,// 然后做阈值的检查,避免过于频繁的image的downloadif (checkRecentImageEnable &&NameNodeFile.IMAGE.equals(parsedParams.getNameNodeFile()) &&timeDelta < checkpointPeriod &&txid - lastCheckpointTxid < checkpointTxnCount) {// only when at least one of two conditions are met we accept// a new fsImage// 1. most recent image's txid is too far behind// 2. last checkpoint time was too oldString message = "Rejecting a fsimage due to small time delta "+ "and txnid delta. Time since previous checkpoint is "+ timeDelta + " expecting at least " + checkpointPeriod+ " txnid delta since previous checkpoint is " +(txid - lastCheckpointTxid) + " expecting at least "+ checkpointTxnCount;LOG.info(message);sendError(response, HttpServletResponse.SC_CONFLICT, message);return null;}try {...InputStream stream = request.getInputStream();try {long start = monotonicNow();//4) 当前NN进行image的downloadMD5Hash downloadImageDigest = TransferFsImage.handleUploadImageRequest(request, txid,nnImage.getStorage(), stream,parsedParams.getFileSize(), getThrottler(conf));nnImage.saveDigestAndRenameCheckpointImage(nnf, txid,downloadImageDigest);...} finally {// 5)NN image下载完成,从请求list中移除当前equestcurrentlyDownloadingCheckpoints.remove(imageRequest);stream.close();}} finally {nnImage.removeFromCheckpointing(txid);}return null;}});} catch (Throwable t) {...}}

从上面代码我们可以看到,Active NN的这部分逻辑会进行upload行为的时间和阈值的检查,避免过于频繁的去下载来自Standby NN的image。

Active/Standby NN间的failover行为


HA服务间的failover行为是另一个需要特别改动的地方,目前HDFS的主从切换依赖的是ZKFC的机制做的。它的一个简单原理是谁率先能够抢到zk的锁并在上面创建相应的znode节点,那么此节点就成为当前的Active服务节点。如果这个znode节点的状态发生改变了,将会触发相应新的Active的选举产生。

我们先来看下原本的failover过程是如何实现的,这样能够方便我们理解多Standby情况的failover实现。

ZKFailoverController.java

  /*** Coordinate a graceful failover. This proceeds in several phases:* 1) Pre-flight checks: ensure that the local node is healthy, and* thus a candidate for failover.* 2) Determine the current active node. If it is the local node, no* need to failover - return success.* 3) Ask that node to yield from the election for a number of seconds.* 4) Allow the normal election path to run in other threads. Wait until* we either become unhealthy or we see an election attempt recorded by* the normal code path.* 5) Allow the old active to rejoin the election, so a future* failback is possible.*/private void doGracefulFailover()throws ServiceFailedException, IOException, InterruptedException {int timeout = FailoverController.getGracefulFenceTimeout(conf) * 2;// Phase 1: pre-flight checkscheckEligibleForFailover();// 1) 获取当前Active地址信息,即老Active信息HAServiceTarget oldActive = getCurrentActive();if (oldActive == null) {// No node is currently active. So, if we aren't already// active ourselves by means of a normal election, then there's// probably something preventing us from becoming active.throw new ServiceFailedException("No other node is currently active.");}// 如果目标target节点已经是active了,则不执行后面的failover逻辑if (oldActive.getAddress().equals(localTarget.getAddress())) {LOG.info("Local node " + localTarget + " is already active. " +"No need to failover. Returning success.");return;}// 2)让当前Active服务在指定时间里退出其Active服务角色,并在此时间之内其不能进行Active服务的竞争获取LOG.info("Asking " + oldActive + " to cede its active state for " +timeout + "ms");ZKFCProtocol oldZkfc = oldActive.getZKFCProxy(conf, timeout);oldZkfc.cedeActive(timeout);// 3)等待当前节点成功变为Active服务,因为没有别的服务竞争Active的选举了ActiveAttemptRecord attempt = waitForActiveAttempt(timeout + 60000);if (attempt == null) {// We didn't even make an attempt to become active.synchronized(this) {if (lastHealthState != State.SERVICE_HEALTHY) {throw new ServiceFailedException("Unable to become active. " +"Service became unhealthy while trying to failover.");          }}throw new ServiceFailedException("Unable to become active. " +"Local node did not get an opportunity to do so from ZooKeeper, " +"or the local node took too long to transition to active.");}// 4)恢复原Active节点的选举权oldZkfc.cedeActive(-1);if (attempt.succeeded) {LOG.info("Successfully became active. " + attempt.status);} else {// Propagate failureString msg = "Failed to become active. " + attempt.status;throw new ServiceFailedException(msg);}}

上面的过程归纳来说是通过短暂剥夺其它服务的Active选举权来完成目标节点Active身份的成功获取。在多Standby的实现中,这套逻辑同样能够完美地适用,只是目标剥夺选举权的对象从一个Standby NN变为多个Standby NN。

相关代码如下,还是ZKFailoverController.java这个类:

  private void doGracefulFailover()throws ServiceFailedException, IOException, InterruptedException {int timeout = FailoverController.getGracefulFenceTimeout(conf) * 2;// Phase 1: pre-flight checkscheckEligibleForFailover();// 1.a)获取当前active节点信息HAServiceTarget oldActive = getCurrentActive();if (oldActive == null) {// No node is currently active. So, if we aren't already// active ourselves by means of a normal election, then there's// probably something preventing us from becoming active.throw new ServiceFailedException("No other node is currently active.");}// 1.b)判断如果当前节点已经是active了,则执行返回if (oldActive.getAddress().equals(localTarget.getAddress())) {LOG.info("Local node " + localTarget + " is already active. " +"No need to failover. Returning success.");return;}// 2.a)获取其它NN节点信息List<HAServiceTarget> otherNodes = getAllOtherNodes();List<ZKFCProtocol> otherZkfcs = new ArrayList<ZKFCProtocol>(otherNodes.size());// 2.b)发起RPC操作,让其它节点在超时时间内放弃Active选择long st = System.nanoTime();HAServiceTarget activeNode = null;for (HAServiceTarget remote : otherNodes) {// same location, same node - may not always be == equalityif (remote.getAddress().equals(oldActive.getAddress())) {activeNode = remote;continue;}otherZkfcs.add(cedeRemoteActive(remote, timeout));}assertactiveNode != null : "Active node does not match any known remote node";// 2.c)让原Active的节点放弃其当前Active身份,此步骤会触发新的Active选举otherZkfcs.add(cedeRemoteActive(activeNode, timeout));// 3)等待当前节点成为新的Active节点,因为其它节点在前面操作中已经被禁止在一定范围时间里被Active选举ActiveAttemptRecord attempt = waitForActiveAttempt(timeout + 60000, st);if (attempt == null) {// We didn't even make an attempt to become active.synchronized(this) {if (lastHealthState != State.SERVICE_HEALTHY) {throw new ServiceFailedException("Unable to become active. " +"Service became unhealthy while trying to failover.");          }}throw new ServiceFailedException("Unable to become active. " +"Local node did not get an opportunity to do so from ZooKeeper, " +"or the local node took too long to transition to active.");}// 4) 当前节点选举成功成为Active后,恢复其它节点的Active选举权for (ZKFCProtocol zkfc : otherZkfcs) {zkfc.cedeActive(-1);}if (attempt.succeeded) {LOG.info("Successfully became active. " + attempt.status);} else {// Propagate failureString msg = "Failed to become active. " + attempt.status;throw new ServiceFailedException(msg);}}

上面操作方法对应的failover场景是admin手动触发failover的行为,但如果遇到因为Active服务不可用导致的自动failover时,就会变成各Standby NN对zk的公平抢锁的选择过程了,谁能够率先拿到zk lock,写入znode信息,它就能成为新的Active NN。鉴于这里可能存在的zk lock的并发竞争,社区不推荐过多Standby NN的部署, 3~5个Standby NN是一个比较理想的多Standby部署模式。

Standby NN向ActiveNN发起的log roll操作行为


最后一部分是Standby NN向Active发起的log roll行为。Standby NN在从JN tail log之前,会向Active NN发起log roll的RPC操作,使其产生一个新的finalized的editlog,然后Standby NN进行最新editlog的tail。

EditLogTailer.java

    private void doWork() {long currentSleepTimeMs = sleepTimeMs;while (shouldRun) {long editsTailed  = 0;try {// There's no point in triggering a log roll if the Standby hasn't// read any more transactions since the last time a roll was// triggered.boolean triggeredLogRoll = false;if (tooLongSinceLastLoad() &&lastRollTriggerTxId < lastLoadedTxnId) {// 1)trigger active NN的log roll操作triggerActiveLogRoll();triggeredLogRoll = true;}/*** Check again in case someone calls {@link EditLogTailer#stop} while* we're triggering an edit log roll, since ipc.Client catches and* ignores {@link InterruptedException} in a few places. This fixes* the bug described in HDFS-2823.*/if (!shouldRun) {break;}// Prevent reading of name system while being modified. The full// name system lock will be acquired to further block even the block// state updates.namesystem.cpLockInterruptibly();long startTime = Time.monotonicNow();try {NameNode.getNameNodeMetrics().addEditLogTailInterval(startTime - lastLoadTimeMs);// 2)editlog的tail操作editsTailed = doTailEdits();} finally {namesystem.cpUnlock();NameNode.getNameNodeMetrics().addEditLogTailTime(Time.monotonicNow() - startTime);}...
}

在这里唯一的改动是log roll时Active NN的寻找,因为Standby NN并不知道哪个是真正的Active NN服务。这里的实现是用一个新的Proxy进行了封装实现,这个Proxy对象在执行实际work方法时时会去找找真正的Active。

  abstract class MultipleNameNodeProxy<T> implements Callable<T> {/*** Do the actual work to the remote namenode via the {@link #cachedActiveProxy}.* @return the result of the work, if there is one* @throws IOException if the actions done to the proxy throw an exception.*/protected abstract T doWork() throws IOException;public T call() throws IOException {// reset the loop count on successnnLoopCount = 0;while ((cachedActiveProxy = getActiveNodeProxy()) != null) {try {T ret = doWork();return ret;} catch (IOException e) {LOG.warn("Exception from remote name node " + currentNN+ ", try next.", e);// 如果当前获取的Proxy对象执行抛错(比如访问到的是Standby NN),// 则重置当前Proxy对象,则继续进行Proxy对象的获取和RPC操作的执行cachedActiveProxy = null;nnLoopCount++;}}throw new IOException("Cannot find any valid remote NN to service request!");}// 遍历得到下一个潜在的Active Proxy对象,此处并不保证一定是Active的Proxy对象private NamenodeProtocol getActiveNodeProxy() throws IOException {if (cachedActiveProxy == null) {while (true) {// if we have reached the max loop count, quit by returning nullif ((nnLoopCount / nnCount) >= maxRetries) {return null;}currentNN = nnLookup.next();try {int rpcTimeout = conf.getInt(DFSConfigKeys.DFS_HA_LOGROLL_RPC_TIMEOUT_KEY,DFSConfigKeys.DFS_HA_LOGROLL_RPC_TIMEOUT_DEFAULT);NamenodeProtocolPB proxy = RPC.waitForProxy(NamenodeProtocolPB.class,RPC.getProtocolVersion(NamenodeProtocolPB.class), currentNN.getIpcAddress(), conf,rpcTimeout, Long.MAX_VALUE);cachedActiveProxy = new NamenodeProtocolTranslatorPB(proxy);break;} catch (IOException e) {LOG.info("Failed to reach " + currentNN, e);// couldn't even reach this NN, try the next onennLoopCount++;}}}assert cachedActiveProxy != null;return cachedActiveProxy;}

Trigger log roll的操作会进行上面这个Proxy的call方法调用:
EditLogTailer.java

  void triggerActiveLogRoll() {LOG.info("Triggering log roll on remote NameNode");Future<Void> future = null;try {future = rollEditsRpcExecutor.submit(getNameNodeProxy());future.get(rollEditsTimeoutMs, TimeUnit.MILLISECONDS);...}/*** NameNodeProxy factory method.* @return a Callable to roll logs on remote NameNode.*/@VisibleForTestingCallable<Void> getNameNodeProxy() {return new MultipleNameNodeProxy<Void>() {@Overrideprotected Void doWork() throws IOException {// Proxy实际工作方法为log roll操作cachedActiveProxy.rollEditLog();return null;}};}

以上就是HDFS支持Multiple Standby所涉及到的改动,整体改动的部分其实并不算很多,现有HA的机制完全能够复用。

相关链接


[1].https://issues.apache.org/jira/browse/HDFS-6440

HDFS Multiple Standby原理分析相关推荐

  1. airflow mysql_Airflow 使用及原理分析

    Airflow 入门及使用 什么是 Airflow?Airflow 是一个使用 Python 语言编写的 Data Pipeline 调度和监控工作流的平台. Airflow 是通过 DAG(Dire ...

  2. 老李推荐:第6章6节《MonkeyRunner源码剖析》Monkey原理分析-事件源-事件源概览-命令队列...

    老李推荐:第6章6节<MonkeyRunner源码剖析>Monkey原理分析-事件源-事件源概览-命令队列 事件源在获得字串命令并把它翻译成对应的MonkeyEvent事件后,会把这些事件 ...

  3. Hadoop生态圈-Zookeeper的工作原理分析

    Hadoop生态圈-Zookeeper的工作原理分析 作者:尹正杰 版权声明:原创作品,谢绝转载!否则将追究法律责任.   无论是是Kafka集群,还是producer和consumer都依赖于Zoo ...

  4. Go Slice实现原理分析

    Go Slice实现原理分析 认识 Slice 一种可变长度的数 操作 make :创建Slice,可以提前分配内存, append:往Slice添加元素 package main import (& ...

  5. 原理剖析(第 012 篇)Netty之无锁队列MpscUnboundedArrayQueue原理分析

    原理剖析(第 012 篇)Netty之无锁队列MpscUnboundedArrayQueue原理分析 - 一.大致介绍 1.了解过netty原理的童鞋,其实应该知道工作线程组的每个子线程都维护了一个任 ...

  6. JPEG原理分析 及 JPEG解码器的调试

    文章目录 数据压缩实验(五) 一.JPEG原理分析 1.概述 优点 缺点 2.JPEG编解码原理 (1)彩色空间 (2)Level offset--零偏置电平下移 (3)8x8 DCT--离散余弦变换 ...

  7. java signature 性能_Java常见bean mapper的性能及原理分析

    背景 在分层的代码架构中,层与层之间的对象避免不了要做很多转换.赋值等操作,这些操作重复且繁琐,于是乎催生出很多工具来优雅,高效地完成这个操作,有BeanUtils.BeanCopier.Dozer. ...

  8. Select函数实现原理分析

    转载自 http://blog.chinaunix.net/uid-20643761-id-1594860.html select需要驱动程序的支持,驱动程序实现fops内的poll函数.select ...

  9. spring ioc原理分析

    spring ioc原理分析 spring ioc 的概念 简单工厂方法 spirng ioc实现原理 spring ioc的概念 ioc: 控制反转 将对象的创建由spring管理.比如,我们以前用 ...

最新文章

  1. bzoj 1040: [ZJOI2008]骑士 树形dp
  2. 机器学习四剑客3——Pandas
  3. php请求接口数据,php curl请求接口并获取数据的示例代码
  4. 自已写的几个分页的存储过程
  5. 看尽SLAM、ReID和文字识别等技术进展!15位视觉技术大咖齐聚一堂
  6. 模态对话框的消息循环原理及分析笔记
  7. 在Salesforce中调用外部系统所提供的的Web Service
  8. 物联网-移远M26模块MQTT开发(AT命令)
  9. STM32F103C8T6用官方bootloader实现IAP升级操作方法
  10. 电力安全工作规程发电厂和变电站电气部分_一招告诉你,何为电力系统
  11. java 查询sql_Java 中如何使用 SQL 查询文本
  12. JavaScript学习之旅-9(原创)
  13. 获取公众号文章封面 API 接口
  14. 西藏拉姆拉错:蓝蓝的湖水
  15. 牛客练习赛87 -C-牛老板
  16. 什么是SCRM 客户scrm管理系统 - whale 帷幄
  17. ExpandableListview的简单使用
  18. 用C语言编写简单的通讯录(大一课程作业)
  19. AWS免费套餐避坑攻略:查看账户下所有region的资源,及时关停不需要的服务
  20. python打开网页被禁止_Python请求无法刮取403禁止的网页

热门文章

  1. HTML+CSS综合实验之简单实现环球互联网产业园网页--渐入HTML4
  2. 强的离谱,如何用Python兼职接单?攻略来袭!大数据推送给即将暴富得人!
  3. matlab 可靠性,matlab可靠度.pdf
  4. 经纬财富:中山套单无需烦恼,解套有待学习
  5. UK助力2019BTC100 新年私享会圆满落幕
  6. 徐州2018年大学计算机比赛,建策杯2018年江苏省大学生计算机设计大赛暨2018年(第11.PDF...
  7. 设计模式中的结构类模式
  8. [免费专栏] Android安全之ADB命令总结「收藏版」
  9. integral函数Opencv源码理解-leetcode动态规划的使用场景
  10. 计算机教育内容怎么写,计算机教育论文摘要怎么写 计算机教育论文摘要范文参考...