flink1.8 基于Zookeeper的HA源碼分析

Zookeeper HA相關(guān)配置

## 使用zk做HA
high-availability: zookeeper
## zk地址
high-availability.zookeeper.quorum: node1:2181,node2:2181,node3:2181
## flink在zk下的工作路徑
high-availability.zookeeper.path.root: /flink
## 任務(wù)所在的HA路徑
high-availability.cluster-id: /default
## 保存元數(shù)據(jù)到文件系統(tǒng)
high-availability.storageDir: hdfs:///flink/recovery

## --任務(wù)運(yùn)行在YARN上的配置--
## applicationMaster重試的次數(shù)灶平,默認(rèn)為1,當(dāng)application master失敗的時(shí)候雅宾,該任務(wù)不會(huì)重啟养涮。
## 設(shè)置一個(gè)比較大的值的話,yarn會(huì)嘗試重啟applicationMaster眉抬。
yarn.application-attempts: 10
## flink是否應(yīng)該重新分配失敗的taskmanager容器贯吓。默認(rèn)是true。
yarn.reallocate-failed:true
## applicationMaster可以接受的容器最大失敗次數(shù)蜀变,達(dá)到這個(gè)參數(shù)悄谐,就會(huì)認(rèn)為yarn job失敗。
## 默認(rèn)這個(gè)次數(shù)和初始化請(qǐng)求的taskmanager數(shù)量相等(-n 參數(shù)指定的)库北。
yarn.maximum-failed-containers:1


flink使用Zookeeper做HA

flink的ResourceManager爬舰、Dispatcher、JobManager寒瓦、WebServer組件都需要高可用保證情屹,同時(shí)flink高可用還需要持久化checkpoint的元數(shù)據(jù)信息,保留最近一次已經(jīng)完成的checkpoint等工作杂腰,其中最重要的就是組件的leader選舉垃你、leader狀態(tài)跟蹤。本次抽取出Flink使用zk實(shí)現(xiàn)leader選舉喂很、leader狀態(tài)跟蹤代碼蜡镶,學(xué)習(xí)下flink是如何使用curator的。類之間的關(guān)系如下:


ZooKeeperHaServices是HighAvailabilityServices基于zookeeper的實(shí)現(xiàn)恤筛,通過(guò)使用ZooKeeperUtils類來(lái)創(chuàng)建組件的LeaderRetrievalService以及LeaderElectionService官还。

LeaderRetrievalService用來(lái)跟蹤leader的變化,當(dāng)發(fā)現(xiàn)leader地址變化時(shí)毒坛,要通知依賴它的組件去依賴新的leader望伦。比如getResourceManagerLeaderRetriever方法,flink會(huì)監(jiān)聽(tīng)zk的/leader/resource_manager_lock節(jié)點(diǎn)內(nèi)容變化煎殷,內(nèi)容是rm的leader地址和leaderUUID,而taskmanger調(diào)用該服務(wù)的start方法傳遞了一個(gè)LeaderRetrievalListener屯伞。如果節(jié)點(diǎn)內(nèi)容發(fā)生變化,意味著rm的leader地址發(fā)生變化,那么的LeaderRetrievalListener的notifyLeaderAddress就會(huì)通知taskmanger去新的ResourceManager地址進(jìn)行注冊(cè)豪直。zk實(shí)現(xiàn)該功能使用的是curator的NodeCache并重寫(xiě)了nodeChanged方法劣摇。

LeaderElectionService用來(lái)進(jìn)行l(wèi)eader選舉工作,當(dāng)節(jié)點(diǎn)成為leader后會(huì)調(diào)用LeaderContender的grantLeadership方法弓乙。以ResourceManagerLeaderElection為例末融,flink會(huì)在zk的/leaderlatch/resource_manager_lock路徑下創(chuàng)建臨時(shí)節(jié)點(diǎn)钧惧,創(chuàng)建成功的rm節(jié)點(diǎn)成為leader觸發(fā)rm的grantLeadership,最終將當(dāng)前地址和UUID寫(xiě)入/leader/resource_manager_lock中勾习,這樣就觸發(fā)了LeaderRetrievalService服務(wù)浓瞪。zk實(shí)現(xiàn)leader選舉使用的是curator的LeaderLatch并重寫(xiě)了isLeader和notLeader方法。同時(shí)使用NodeCache監(jiān)聽(tīng)/leader/resource_manager_lock內(nèi)容變化巧婶,確保新leader地址和UUID成功寫(xiě)入節(jié)點(diǎn)乾颁。

LeaderRetrievalListener對(duì)LeaderRetrievalService的leader地址變化做出響應(yīng),通過(guò)notifyLeaderAddress傳遞新leader地址艺栈。

LeaderContender對(duì)LeaderElectionService的節(jié)點(diǎn)角色發(fā)生變化做出響應(yīng)英岭,通過(guò)grantLeadership和revokeLeadership進(jìn)行l(wèi)eader的授權(quán)和撤銷工作。

一個(gè)集群目錄下的zk結(jié)構(gòu)如下圖所示:


zk文件系統(tǒng)目錄

flink相關(guān)源碼

簡(jiǎn)單的走一下流程湿右,看看集群?jiǎn)?dòng)時(shí)是如何創(chuàng)建ZooKeeperHaServices的诅妹。

集群?jiǎn)?dòng)入口ClusterEntrypoint
  • 根據(jù)集群的部署模式session or perjob由對(duì)應(yīng)的子類調(diào)用ClusterEntrypoint的startCluster方法啟動(dòng)集群,接著會(huì)先調(diào)用initializeServices方法诅需,啟動(dòng)集群相關(guān)的組件信息漾唉。這里只看啟動(dòng)haServices部分。
public void startCluster() throws ClusterEntrypointException {

  SecurityContext securityContext = installSecurityContext(configuration);
  securityContext.runSecured((Callable<Void>) () -> {
        runCluster(configuration);
         return null;
  });
}

protected void initializeServices(Configuration configuration) {

    ioExecutor = Executors.newFixedThreadPool(
        Hardware.getNumberCPUCores(),
        new ExecutorThreadFactory("cluster-io"));
        
    haServices = createHaServices(configuration, ioExecutor);
    
    blobServer = new BlobServer(configuration, haServices.createBlobStore());
    blobServer.start();
        
    }
}
  • 根據(jù)high-availability配置創(chuàng)建ZooKeeperHaServices,默認(rèn)情況下為NONE堰塌。
protected HighAvailabilityServices createHaServices(
    Configuration configuration,
    Executor executor) throws Exception {
    //創(chuàng)建HA服務(wù)時(shí)不需要地址解析
    return HighAvailabilityServicesUtils.createHighAvailabilityServices(
        configuration,
        executor,
        HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
}


//根據(jù)傳遞的high-availability配置赵刑,選擇創(chuàng)建哪種HA服務(wù),默認(rèn)為NONE
public static HighAvailabilityServices createHighAvailabilityServices(
    Configuration configuration,
    Executor executor,
    AddressResolution addressResolution) throws Exception {
    //獲取high-availability配置 如:zookeeper
    HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(configuration);

    switch (highAvailabilityMode) {
        case NONE:
            final Tuple2<String, Integer> hostnamePort = getJobManagerAddress(configuration);

            final String jobManagerRpcUrl = AkkaRpcServiceUtils.getRpcUrl(
                hostnamePort.f0,
                hostnamePort.f1,
                JobMaster.JOB_MANAGER_NAME,
                addressResolution,
                configuration);
            final String resourceManagerRpcUrl = AkkaRpcServiceUtils.getRpcUrl(
                hostnamePort.f0,
                hostnamePort.f1,
                ResourceManager.RESOURCE_MANAGER_NAME,
                addressResolution,
                configuration);
            final String dispatcherRpcUrl = AkkaRpcServiceUtils.getRpcUrl(
                hostnamePort.f0,
                hostnamePort.f1,
                Dispatcher.DISPATCHER_NAME,
                addressResolution,
                configuration);

            final String address = checkNotNull(configuration.getString(RestOptions.ADDRESS),
                "%s must be set",
                RestOptions.ADDRESS.key());
            final int port = configuration.getInteger(RestOptions.PORT);
            final boolean enableSSL = SSLUtils.isRestSSLEnabled(configuration);
            final String protocol = enableSSL ? "https://" : "http://";

            return new StandaloneHaServices(
                resourceManagerRpcUrl,
                dispatcherRpcUrl,
                jobManagerRpcUrl,
                String.format("%s%s:%s", protocol, address, port));
        case ZOOKEEPER:
            //元數(shù)據(jù)存儲(chǔ)服務(wù)  我們通常使用FileSystemBlobStore 路徑就是 high-availability.storageDir: hdfs:///flink/recovery
            BlobStoreService blobStoreService = BlobUtils.createBlobStoreFromConfig(configuration);
            //  使用 ZooKeeper做HA服務(wù)
            return new ZooKeeperHaServices(
                ZooKeeperUtils.startCuratorFramework(configuration),
                executor,
                configuration,
                blobStoreService);

        case FACTORY_CLASS:
            return createCustomHAServices(configuration, executor);

        default:
            throw new Exception("Recovery mode " + highAvailabilityMode + " is not supported.");
    }
}
  • ZooKeeperHaServices主要提供了創(chuàng)建LeaderRetrievalService和LeaderElectionService方法场刑,并給出了各個(gè)服務(wù)組件使用的ZK節(jié)點(diǎn)名稱般此。別看是以_lock結(jié)尾,這個(gè)節(jié)點(diǎn)名稱既在leaderlatcher做leader選舉的分布式鎖產(chǎn)生的路徑牵现,又在leader目錄下用來(lái)存放leader的地址信息铐懊。

private static final String RESOURCE_MANAGER_LEADER_PATH = "/resource_manager_lock";
private static final String DISPATCHER_LEADER_PATH = "/dispatcher_lock";
private static final String JOB_MANAGER_LEADER_PATH = "/job_manager_lock";
//  web展示服務(wù)
private static final String REST_SERVER_LEADER_PATH = "/rest_server_lock";

// 創(chuàng)建ResourceManagerLeaderRetriever,對(duì)RM的leader地址變化進(jìn)行跟蹤
public LeaderRetrievalService getResourceManagerLeaderRetriever() {
    return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
}

// 創(chuàng)建ResourceManagerLeaderElectionService瞎疼,對(duì)RMleader掛掉后重新進(jìn)行選舉
public LeaderElectionService getResourceManagerLeaderElectionService() {
    return ZooKeeperUtils.createLeaderElectionService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
}
  • ZooKeeperUtils創(chuàng)建LeaderRetrievalService流程科乎。
  1. 接收curator客戶端以及服務(wù)在zk下的節(jié)點(diǎn)路徑,創(chuàng)建出ZooKeeperLeaderRetrievalService(ZKLRS)對(duì)象贼急。
  2. ZKLRS這個(gè)對(duì)象就是對(duì)zk節(jié)點(diǎn)的內(nèi)容進(jìn)行了監(jiān)聽(tīng)茅茂,當(dāng)內(nèi)容發(fā)生變化時(shí),通知給通過(guò)start方法傳遞過(guò)來(lái)的LeaderRetrievalListener太抓。
public void start(LeaderRetrievalListener listener) throws Exception {
    synchronized (lock) {
        //leader發(fā)生變化時(shí)空闲,通知對(duì)應(yīng)的LeaderRetrievalListener
        leaderListener = listener;
        // 異常時(shí)調(diào)用當(dāng)前對(duì)象的unhandledError方法
        client.getUnhandledErrorListenable().addListener(this);
        // 使用NodeCache監(jiān)聽(tīng)節(jié)點(diǎn)內(nèi)容變化
        cache.getListenable().addListener(this);
        cache.start();
        
        // 對(duì)會(huì)話連接狀態(tài)進(jìn)行跟蹤
        client.getConnectionStateListenable().addListener(connectionStateListener);

        running = true;
    }
}
  1. 通過(guò)重寫(xiě)nodeChanged方法,來(lái)獲取Leader變更后的地址走敌,并傳遞新的地址
public void nodeChanged() throws Exception {
    synchronized (lock) {
        if (running) {
            try {
                LOG.debug("Leader node has changed.");

                ChildData childData = cache.getCurrentData();

                String leaderAddress;
                UUID leaderSessionID;

                if (childData == null) {
                    leaderAddress = null;
                    leaderSessionID = null;
                } else {
                    byte[] data = childData.getData();

                    if (data == null || data.length == 0) {
                        leaderAddress = null;
                        leaderSessionID = null;
                    } else {
                        ByteArrayInputStream bais = new ByteArrayInputStream(data);
                        ObjectInputStream ois = new ObjectInputStream(bais);
                        // leader 地址
                        leaderAddress = ois.readUTF();
                        // leader uuid
                        leaderSessionID = (UUID) ois.readObject();
                    }
                }
                // leader 地址發(fā)生變化
                if (!(Objects.equals(leaderAddress, lastLeaderAddress) &&
                    Objects.equals(leaderSessionID, lastLeaderSessionID))) {

                    lastLeaderAddress = leaderAddress;
                    lastLeaderSessionID = leaderSessionID;
                    //  傳遞新的leaderAddress和leaderSessionID
                    leaderListener.notifyLeaderAddress(leaderAddress, leaderSessionID);
                }
            } catch (Exception e) {
                leaderListener.handleError(new Exception("Could not handle node changed event.", e));
                throw e;
            }
        } else {
            LOG.debug("Ignoring node change notification since the service has already been stopped.");
        }
    }
}
  • ZooKeeperUtils創(chuàng)建ZooKeeperLeaderElectionService流程碴倾。
  1. 傳遞leader所在的zk路徑、選舉時(shí)臨時(shí)節(jié)點(diǎn)創(chuàng)建的zk路徑。之所以要傳遞leader節(jié)點(diǎn)是要在新leader產(chǎn)生時(shí)跌榔,將新leader的地址和uuid寫(xiě)入异雁。
public static ZooKeeperLeaderElectionService createLeaderElectionService(
        final CuratorFramework client,
        final Configuration configuration,
        final String pathSuffix) {
    // 在leaderlatch節(jié)點(diǎn)下進(jìn)行選舉
    final String latchPath = configuration.getString(
        HighAvailabilityOptions.HA_ZOOKEEPER_LATCH_PATH) + pathSuffix;
     // leader節(jié)點(diǎn)    
    final String leaderPath = configuration.getString(
        HighAvailabilityOptions.HA_ZOOKEEPER_LEADER_PATH) + pathSuffix;

    return new ZooKeeperLeaderElectionService(client, latchPath, leaderPath);
}
  1. 通過(guò)調(diào)用start方法傳遞LeaderContender,并開(kāi)啟leader選舉矫户。
public void start(LeaderContender contender) throws Exception {

    synchronized (lock) {
        // 綁定異常處理監(jiān)聽(tīng)器
        client.getUnhandledErrorListenable().addListener(this);
        // 傳遞Contender競(jìng)爭(zhēng)者
        leaderContender = contender;
        
        //開(kāi)啟leader選舉服務(wù)片迅,成為leader的節(jié)點(diǎn)會(huì)觸發(fā)isleader
        leaderLatch.addListener(this);
        leaderLatch.start();
        
        //監(jiān)聽(tīng)leader節(jié)點(diǎn)內(nèi)容變化
        cache.getListenable().addListener(this);
        cache.start();

        client.getConnectionStateListenable().addListener(listener);

        running = true;
    }
}
  1. 當(dāng)某一Contender成為leader后残邀,會(huì)觸發(fā)grantLeadership傳遞新leader的uuid進(jìn)行授權(quán)皆辽,并調(diào)用LeaderElectionService的confirmLeaderSessionID,將新leader地址寫(xiě)入leader節(jié)點(diǎn)芥挣。
public void confirmLeaderSessionID(UUID leaderSessionID) {
    // 是Leader
    if (leaderLatch.hasLeadership()) {
        // check if this is an old confirmation call
        synchronized (lock) {
            if (running) {
                if (leaderSessionID.equals(this.issuedLeaderSessionID)) {
                    confirmedLeaderSessionID = leaderSessionID;
                    // 將confirmLeaderSessionID寫(xiě)到 leader目錄下
                    writeLeaderInformation(confirmedLeaderSessionID);
                }
            }
        }
    } 
}
  1. 寫(xiě)入時(shí)會(huì)觸發(fā)當(dāng)前對(duì)象的nodeChanged方法驱闷,該方法用來(lái)確保新leader地址和uuid成功寫(xiě)入。
public void nodeChanged() throws Exception {
    try {
        // leaderSessionID is null if the leader contender has not yet confirmed the session ID
        if (leaderLatch.hasLeadership()) { // leader
            synchronized (lock) {
                if (running) {
                    // 當(dāng)選為leader 已經(jīng)被確認(rèn)
                    if (confirmedLeaderSessionID != null) {
                        ChildData childData = cache.getCurrentData();
                        //  沒(méi)寫(xiě)進(jìn)去空免,再寫(xiě)一次
                        if (childData == null) {
                            if (LOG.isDebugEnabled()) {
                                LOG.debug(
                                    "Writing leader information into empty node by {}.",
                                    leaderContender.getAddress());
                            }
                            writeLeaderInformation(confirmedLeaderSessionID);
                        } else {
                            byte[] data = childData.getData();

                            if (data == null || data.length == 0) {
                                // the data field seems to be empty, rewrite information
                                writeLeaderInformation(confirmedLeaderSessionID);
                            } else {
                                ByteArrayInputStream bais = new ByteArrayInputStream(data);
                                ObjectInputStream ois = new ObjectInputStream(bais);

                                String leaderAddress = ois.readUTF();
                                UUID leaderSessionID = (UUID) ois.readObject();

                                if (!leaderAddress.equals(this.leaderContender.getAddress()) ||
                                    (leaderSessionID == null || !leaderSessionID.equals(confirmedLeaderSessionID))) {
                                    writeLeaderInformation(confirmedLeaderSessionID);
                                }
                            }
                        }
                    }
                } else {
                    // leader未確認(rèn)confirmedLeaderSessionID
                    LOG.debug("Ignoring node change notification since the service has already been stopped.");
                }
            }
        }
    } catch (Exception e) {
            ...
    }
}

writeLeaderInformation用來(lái)寫(xiě)入leader地址和uuid,寫(xiě)入時(shí)先判斷l(xiāng)eader節(jié)點(diǎn)是否由當(dāng)前l(fā)eader會(huì)話創(chuàng)建的空另,如果不是則刪除后重寫(xiě)創(chuàng)建。

protected void writeLeaderInformation(UUID leaderSessionID) {
    try {
        ByteArrayOutputStream baos = new ByteArrayOutputStream();
        ObjectOutputStream oos = new ObjectOutputStream(baos);
        //  leader 地址
        oos.writeUTF(leaderContender.getAddress());
        //  leader的 UUID
        oos.writeObject(leaderSessionID);

        oos.close();

        boolean dataWritten = false;

        while (!dataWritten && leaderLatch.hasLeadership()) {
            Stat stat = client.checkExists().forPath(leaderPath);

            if (stat != null) {
                long owner = stat.getEphemeralOwner();
                long sessionID = client.getZookeeperClient().getZooKeeper().getSessionId();
                //節(jié)點(diǎn)由當(dāng)前會(huì)話創(chuàng)建
                if (owner == sessionID) {
                    try {
                        client.setData().forPath(leaderPath, baos.toByteArray());

                        dataWritten = true;
                    } catch (KeeperException.NoNodeException noNode) {
                        // node was deleted in the meantime
                    }
                } else {
                    try {
                        //  不是當(dāng)前節(jié)點(diǎn)創(chuàng)建則先刪除
                        client.delete().forPath(leaderPath);
                    } catch (KeeperException.NoNodeException noNode) {
                        // node was deleted in the meantime --> try again
                    }
                }
            } else {
                try {
                    client.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(
                            leaderPath,
                            baos.toByteArray());

                    dataWritten = true;
                } catch (KeeperException.NodeExistsException nodeExists) {
                    // node has been created in the meantime --> try again
                }
            }
        }
    } 
}

本次學(xué)習(xí)了flink如何使用curator來(lái)操作zk節(jié)點(diǎn)蹋砚,實(shí)現(xiàn)leader選舉和leader狀態(tài)跟蹤扼菠。LeaderRetrievalListener和LeaderContender兩個(gè)接口更像是這一部分功能的輸入和輸出,來(lái)跟蹤leader的變化情況坝咐。而中間部分對(duì)zk節(jié)點(diǎn)的操作和狀態(tài)監(jiān)聽(tīng)循榆,則可以抽取出來(lái)在自己的項(xiàng)目中使用。

最后編輯于
?著作權(quán)歸作者所有,轉(zhuǎn)載或內(nèi)容合作請(qǐng)聯(lián)系作者
  • 序言:七十年代末墨坚,一起剝皮案震驚了整個(gè)濱河市秧饮,隨后出現(xiàn)的幾起案子,更是在濱河造成了極大的恐慌泽篮,老刑警劉巖盗尸,帶你破解...
    沈念sama閱讀 217,185評(píng)論 6 503
  • 序言:濱河連續(xù)發(fā)生了三起死亡事件,死亡現(xiàn)場(chǎng)離奇詭異帽撑,居然都是意外死亡泼各,警方通過(guò)查閱死者的電腦和手機(jī),發(fā)現(xiàn)死者居然都...
    沈念sama閱讀 92,652評(píng)論 3 393
  • 文/潘曉璐 我一進(jìn)店門(mén)亏拉,熙熙樓的掌柜王于貴愁眉苦臉地迎上來(lái)扣蜻,“玉大人,你說(shuō)我怎么就攤上這事专筷∪踉簦” “怎么了?”我有些...
    開(kāi)封第一講書(shū)人閱讀 163,524評(píng)論 0 353
  • 文/不壞的土叔 我叫張陵磷蛹,是天一觀的道長(zhǎng)吮旅。 經(jīng)常有香客問(wèn)我,道長(zhǎng),這世上最難降的妖魔是什么庇勃? 我笑而不...
    開(kāi)封第一講書(shū)人閱讀 58,339評(píng)論 1 293
  • 正文 為了忘掉前任檬嘀,我火速辦了婚禮,結(jié)果婚禮上责嚷,老公的妹妹穿的比我還像新娘鸳兽。我一直安慰自己,他們只是感情好罕拂,可當(dāng)我...
    茶點(diǎn)故事閱讀 67,387評(píng)論 6 391
  • 文/花漫 我一把揭開(kāi)白布揍异。 她就那樣靜靜地躺著,像睡著了一般爆班。 火紅的嫁衣襯著肌膚如雪衷掷。 梳的紋絲不亂的頭發(fā)上,一...
    開(kāi)封第一講書(shū)人閱讀 51,287評(píng)論 1 301
  • 那天柿菩,我揣著相機(jī)與錄音戚嗅,去河邊找鬼。 笑死枢舶,一個(gè)胖子當(dāng)著我的面吹牛懦胞,可吹牛的內(nèi)容都是我干的。 我是一名探鬼主播凉泄,決...
    沈念sama閱讀 40,130評(píng)論 3 418
  • 文/蒼蘭香墨 我猛地睜開(kāi)眼躏尉,長(zhǎng)吁一口氣:“原來(lái)是場(chǎng)噩夢(mèng)啊……” “哼!你這毒婦竟也來(lái)了旧困?” 一聲冷哼從身側(cè)響起醇份,我...
    開(kāi)封第一講書(shū)人閱讀 38,985評(píng)論 0 275
  • 序言:老撾萬(wàn)榮一對(duì)情侶失蹤,失蹤者是張志新(化名)和其女友劉穎吼具,沒(méi)想到半個(gè)月后僚纷,有當(dāng)?shù)厝嗽跇?shù)林里發(fā)現(xiàn)了一具尸體,經(jīng)...
    沈念sama閱讀 45,420評(píng)論 1 313
  • 正文 獨(dú)居荒郊野嶺守林人離奇死亡拗盒,尸身上長(zhǎng)有42處帶血的膿包…… 初始之章·張勛 以下內(nèi)容為張勛視角 年9月15日...
    茶點(diǎn)故事閱讀 37,617評(píng)論 3 334
  • 正文 我和宋清朗相戀三年怖竭,在試婚紗的時(shí)候發(fā)現(xiàn)自己被綠了。 大學(xué)時(shí)的朋友給我發(fā)了我未婚夫和他白月光在一起吃飯的照片陡蝇。...
    茶點(diǎn)故事閱讀 39,779評(píng)論 1 348
  • 序言:一個(gè)原本活蹦亂跳的男人離奇死亡痊臭,死狀恐怖,靈堂內(nèi)的尸體忽然破棺而出登夫,到底是詐尸還是另有隱情广匙,我是刑警寧澤,帶...
    沈念sama閱讀 35,477評(píng)論 5 345
  • 正文 年R本政府宣布恼策,位于F島的核電站鸦致,受9級(jí)特大地震影響,放射性物質(zhì)發(fā)生泄漏。R本人自食惡果不足惜分唾,卻給世界環(huán)境...
    茶點(diǎn)故事閱讀 41,088評(píng)論 3 328
  • 文/蒙蒙 一抗碰、第九天 我趴在偏房一處隱蔽的房頂上張望。 院中可真熱鬧绽乔,春花似錦弧蝇、人聲如沸。這莊子的主人今日做“春日...
    開(kāi)封第一講書(shū)人閱讀 31,716評(píng)論 0 22
  • 文/蒼蘭香墨 我抬頭看了看天上的太陽(yáng)。三九已至鞍爱,卻和暖如春鹃觉,著一層夾襖步出監(jiān)牢的瞬間专酗,已是汗流浹背睹逃。 一陣腳步聲響...
    開(kāi)封第一講書(shū)人閱讀 32,857評(píng)論 1 269
  • 我被黑心中介騙來(lái)泰國(guó)打工, 沒(méi)想到剛下飛機(jī)就差點(diǎn)兒被人妖公主榨干…… 1. 我叫王不留祷肯,地道東北人沉填。 一個(gè)月前我還...
    沈念sama閱讀 47,876評(píng)論 2 370
  • 正文 我出身青樓,卻偏偏與公主長(zhǎng)得像佑笋,于是被迫代替她去往敵國(guó)和親翼闹。 傳聞我的和親對(duì)象是個(gè)殘疾皇子,可洞房花燭夜當(dāng)晚...
    茶點(diǎn)故事閱讀 44,700評(píng)論 2 354

推薦閱讀更多精彩內(nèi)容

  • 摘要 要學(xué)習(xí)系統(tǒng)構(gòu)架蒋纬,ZooKeeper (下文簡(jiǎn)稱zk)是無(wú)法繞開(kāi)的開(kāi)源技術(shù)猎荠。大型網(wǎng)站后臺(tái)成百上千的分布式服務(wù)節(jié)...
    老吳學(xué)技術(shù)閱讀 1,330評(píng)論 0 4
  • ... 一、相關(guān)概念 中間件:為分布式系統(tǒng)提供協(xié)調(diào)服務(wù)的組件蜀备,如專門(mén)用于計(jì)算服務(wù)的機(jī)器就是一個(gè)計(jì)算型中間件关摇,還有專...
    帥可兒妞閱讀 475評(píng)論 0 0
  • 一、系統(tǒng)模型 1碾阁、數(shù)據(jù)模型 zk結(jié)構(gòu)視圖與unix的文件系統(tǒng)有點(diǎn)類似输虱,但是沒(méi)有目錄和文件的相關(guān)概念。而是使用特有的...
    小manong閱讀 2,128評(píng)論 0 3
  • 學(xué)習(xí)整理: zookeeper有哪些應(yīng)用脂凶,HA熱備宪睹、分布式鎖、集中配置用到了zookeeper的什么蚕钦? ZooKe...
    大佛愛(ài)讀書(shū)閱讀 231評(píng)論 0 0
  • 畢業(yè)一年多了亭病,我們見(jiàn)面的次數(shù)屈指可數(shù)。匆匆一見(jiàn)嘶居,想說(shuō)的話都沒(méi)有說(shuō)完罪帖,想時(shí)間過(guò)得慢一點(diǎn)。 我在哏都與...
    總有不期而遇的溫暖閱讀 251評(píng)論 0 0